Browse Source

YARN-4447. Provide a mechanism to represent complex filters and parse them at the REST layer (Varun Saxena via sjlee)

Sangjin Lee 9 years ago
parent
commit
089caf49fe
21 changed files with 3437 additions and 192 deletions
  1. 34 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParseConstants.java
  2. 36 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParseException.java
  3. 37 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParser.java
  4. 300 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForCompareExpr.java
  5. 95 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForDataToRetrieve.java
  6. 343 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForEqualityExpr.java
  7. 51 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForExistFilters.java
  8. 78 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForKVFilters.java
  9. 72 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForNumericFilters.java
  10. 71 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForRelationFilters.java
  11. 196 24
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java
  12. 65 131
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
  13. 69 4
      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
  14. 47 2
      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
  15. 36 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
  16. 13 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
  17. 58 3
      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
  18. 37 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
  19. 7 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java
  20. 869 21
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
  21. 923 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesUtils.java

+ 34 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParseConstants.java

@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.timelineservice.reader;
+
+/**
+ * Set of constants used while parsing filter expressions.
+ */
+final class TimelineParseConstants {
+  private TimelineParseConstants() {
+  }
+  static final String COMMA_DELIMITER = ",";
+  static final String COLON_DELIMITER = ":";
+  static final char NOT_CHAR = '!';
+  static final char SPACE_CHAR = ' ';
+  static final char OPENING_BRACKET_CHAR = '(';
+  static final char CLOSING_BRACKET_CHAR = ')';
+  static final char COMMA_CHAR = ',';
+}

+ 36 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParseException.java

@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.timelineservice.reader;
+
+/**
+ * Exception thrown to indicate that a timeline filter expression cannot be
+ * parsed.
+ */
+class TimelineParseException extends Exception {
+
+  private static final long serialVersionUID = 1L;
+
+  public TimelineParseException() {
+    super();
+  }
+
+  public TimelineParseException(String message) {
+    super(message);
+  }
+}

+ 37 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParser.java

@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.timelineservice.reader;
+
+import java.io.Closeable;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+
+@Private
+@Unstable
+interface TimelineParser extends Closeable {
+  /**
+   * Method used for parsing.
+   *
+   * @return a {@link TimelineFilterList} object.
+   * @throws TimelineParseException if any problem occurs while parsing.
+   */
+  TimelineFilterList parse() throws TimelineParseException;
+}

+ 300 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForCompareExpr.java

@@ -0,0 +1,300 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.timelineservice.reader;
+
+import java.util.Deque;
+import java.util.LinkedList;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList.Operator;
+
+/**
+ * Abstract class for parsing compare expressions.
+ * Compare expressions are of the form :
+ * (<key> <compareop> <value>) <op> (<key
+ * > <compareop> <value>)
+ * compareop is used to compare value of a the specified key in the backend
+ * storage. compareop can be :
+ * 1. eq - Equals
+ * 2. ne - Not equals (matches if key does not exist)
+ * 3. ene - Exists and not equals (key must exist for match to occur)
+ * 4. lt - Less than
+ * 5. gt - Greater than
+ * 6. le - Less than or equals
+ * 7. ge - Greater than or equals
+ * compareop's supported would depend on implementation. For instance, all
+ * the above compareops' will be supported for metric filters but only eq,ne and
+ * ene would be supported for KV filters like config/info filters.
+ *
+ * op is a logical operator and can be either AND or OR.
+ *
+ * The way values will be interpreted would also depend on implementation
+ *
+ * A typical compare expression would look as under:
+ * ((key1 eq val1 OR key2 ne val2) AND (key5 gt val45))
+ */
+@Private
+@Unstable
+abstract class TimelineParserForCompareExpr implements TimelineParser {
+  private enum ParseState {
+    PARSING_KEY,
+    PARSING_VALUE,
+    PARSING_OP,
+    PARSING_COMPAREOP
+  }
+  // Main expression.
+  private final String expr;
+  // Expression in lower case.
+  private final String exprInLowerCase;
+  private final String exprName;
+  private int offset = 0;
+  private int kvStartOffset = 0;
+  private final int exprLength;
+  private ParseState currentParseState = ParseState.PARSING_KEY;
+  // Linked list implemented as a stack.
+  private Deque<TimelineFilterList> filterListStack = new LinkedList<>();
+  private TimelineFilter currentFilter = null;
+  private TimelineFilterList filterList = null;
+  public TimelineParserForCompareExpr(String expression, String name) {
+    if (expression != null) {
+      expr = expression.trim();
+      exprLength = expr.length();
+      exprInLowerCase = expr.toLowerCase();
+    } else {
+      expr = null;
+      exprInLowerCase = null;
+      exprLength = 0;
+    }
+    this.exprName = name;
+  }
+
+  protected TimelineFilter getCurrentFilter() {
+    return currentFilter;
+  }
+
+  protected TimelineFilter getFilterList() {
+    return filterList;
+  }
+
+  protected abstract TimelineFilter createFilter();
+
+  protected abstract Object parseValue(String strValue)
+      throws TimelineParseException;
+
+  protected abstract void setCompareOpToCurrentFilter(
+      TimelineCompareOp compareOp, boolean keyMustExistFlag)
+      throws TimelineParseException;
+
+  protected abstract void setValueToCurrentFilter(Object value);
+
+  private void handleSpaceChar() throws TimelineParseException {
+    if (currentParseState == ParseState.PARSING_KEY ||
+        currentParseState == ParseState.PARSING_VALUE) {
+      if (kvStartOffset == offset) {
+        kvStartOffset++;
+        offset++;
+        return;
+      }
+      String str = expr.substring(kvStartOffset, offset);
+      if (currentParseState == ParseState.PARSING_KEY) {
+        if (currentFilter == null) {
+          currentFilter = createFilter();
+        }
+        ((TimelineCompareFilter)currentFilter).setKey(str);
+        currentParseState = ParseState.PARSING_COMPAREOP;
+      } else if (currentParseState == ParseState.PARSING_VALUE) {
+        if (currentFilter != null) {
+          setValueToCurrentFilter(parseValue(str));
+        }
+        currentParseState = ParseState.PARSING_OP;
+      }
+    }
+    offset++;
+  }
+
+  private void handleOpeningBracketChar() throws TimelineParseException {
+    if (currentParseState != ParseState.PARSING_KEY) {
+      throw new TimelineParseException("Encountered unexpected opening " +
+          "bracket while parsing " + exprName + ".");
+    }
+    offset++;
+    kvStartOffset = offset;
+    filterListStack.push(filterList);
+    filterList = null;
+  }
+
+  private void handleClosingBracketChar() throws TimelineParseException {
+    if (currentParseState != ParseState.PARSING_VALUE &&
+        currentParseState != ParseState.PARSING_OP) {
+      throw new TimelineParseException("Encountered unexpected closing " +
+          "bracket while parsing " + exprName + ".");
+    }
+    if (!filterListStack.isEmpty()) {
+      if (currentParseState == ParseState.PARSING_VALUE) {
+        setValueToCurrentFilter(
+            parseValue(expr.substring(kvStartOffset, offset)));
+        currentParseState = ParseState.PARSING_OP;
+      }
+      if (currentFilter != null) {
+        filterList.addFilter(currentFilter);
+      }
+      // As bracket is closing, pop the filter list from top of the stack and
+      // combine it with current filter list.
+      TimelineFilterList fList = filterListStack.pop();
+      if (fList != null) {
+        fList.addFilter(filterList);
+        filterList = fList;
+      }
+      currentFilter = null;
+      offset++;
+      kvStartOffset = offset;
+    } else {
+      throw new TimelineParseException("Encountered unexpected closing " +
+          "bracket while parsing " + exprName + ".");
+    }
+  }
+
+  private void parseCompareOp() throws TimelineParseException {
+    if (offset + 2 >= exprLength) {
+      throw new TimelineParseException("Compare op cannot be parsed for " +
+          exprName + ".");
+    }
+    TimelineCompareOp compareOp = null;
+    boolean keyExistFlag = true;
+    if (expr.charAt(offset + 2) == TimelineParseConstants.SPACE_CHAR) {
+      if (exprInLowerCase.startsWith("eq", offset)) {
+        compareOp = TimelineCompareOp.EQUAL;
+      } else if (exprInLowerCase.startsWith("ne", offset)) {
+        compareOp = TimelineCompareOp.NOT_EQUAL;
+        keyExistFlag = false;
+      } else if (exprInLowerCase.startsWith("lt", offset)) {
+        compareOp = TimelineCompareOp.LESS_THAN;
+      } else if (exprInLowerCase.startsWith("le", offset)) {
+        compareOp = TimelineCompareOp.LESS_OR_EQUAL;
+      } else if (exprInLowerCase.startsWith("gt", offset)) {
+        compareOp = TimelineCompareOp.GREATER_THAN;
+      } else if (exprInLowerCase.startsWith("ge", offset)) {
+        compareOp = TimelineCompareOp.GREATER_OR_EQUAL;
+      }
+      offset = offset + 3;
+    } else if (exprInLowerCase.startsWith("ene ", offset)) {
+      // Not equal but key should be present.
+      compareOp = TimelineCompareOp.NOT_EQUAL;
+      offset = offset + 4;
+    }
+    if (compareOp == null) {
+      throw new TimelineParseException("Compare op cannot be parsed for " +
+          exprName + ".");
+    }
+    setCompareOpToCurrentFilter(compareOp, keyExistFlag);
+    kvStartOffset = offset;
+    currentParseState = ParseState.PARSING_VALUE;
+  }
+
+  private void parseOp(boolean closingBracket) throws TimelineParseException {
+    Operator operator = null;
+    if (exprInLowerCase.startsWith("or ", offset)) {
+      operator = Operator.OR;
+      offset = offset + 3;
+    } else if (exprInLowerCase.startsWith("and ", offset)) {
+      operator = Operator.AND;
+      offset = offset + 4;
+    }
+    if (operator == null) {
+      throw new TimelineParseException("Operator cannot be parsed for " +
+          exprName + ".");
+    }
+    if (filterList == null) {
+      filterList = new TimelineFilterList(operator);
+    }
+    if (currentFilter != null) {
+      filterList.addFilter(currentFilter);
+    }
+    if (closingBracket || filterList.getOperator() != operator) {
+      filterList = new TimelineFilterList(operator, filterList);
+    }
+    currentFilter = null;
+    kvStartOffset = offset;
+    currentParseState = ParseState.PARSING_KEY;
+  }
+
+  @Override
+  public TimelineFilterList parse() throws TimelineParseException {
+    if (expr == null || exprLength == 0) {
+      return null;
+    }
+    boolean closingBracket = false;
+    while (offset < exprLength) {
+      char offsetChar = expr.charAt(offset);
+      switch(offsetChar) {
+      case TimelineParseConstants.SPACE_CHAR:
+        handleSpaceChar();
+        break;
+      case TimelineParseConstants.OPENING_BRACKET_CHAR:
+        handleOpeningBracketChar();
+        break;
+      case TimelineParseConstants.CLOSING_BRACKET_CHAR:
+        handleClosingBracketChar();
+        closingBracket = true;
+        break;
+      default: // other characters.
+        // Parse based on state.
+        if (currentParseState == ParseState.PARSING_COMPAREOP) {
+          parseCompareOp();
+        } else if (currentParseState == ParseState.PARSING_OP) {
+          parseOp(closingBracket);
+          closingBracket = false;
+        } else {
+          // Might be a key or value. Move ahead.
+          offset++;
+        }
+        break;
+      }
+    }
+    if (!filterListStack.isEmpty()) {
+      filterListStack.clear();
+      throw new TimelineParseException("Encountered improper brackets while " +
+          "parsing " + exprName + ".");
+    }
+    if (currentParseState == ParseState.PARSING_VALUE) {
+      setValueToCurrentFilter(
+          parseValue(expr.substring(kvStartOffset, offset)));
+    }
+    if (filterList == null || filterList.getFilterList().isEmpty()) {
+      filterList = new TimelineFilterList(currentFilter);
+    } else if (currentFilter != null) {
+      filterList.addFilter(currentFilter);
+    }
+    return filterList;
+  }
+
+  @Override
+  public void close() {
+    if (filterListStack != null) {
+      filterListStack.clear();
+    }
+    filterList = null;
+    currentFilter = null;
+  }
+}

+ 95 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForDataToRetrieve.java

@@ -0,0 +1,95 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.timelineservice.reader;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList.Operator;
+
+/**
+ * Used for parsing metrics or configs to retrieve.
+ */
+@Private
+@Unstable
+public class TimelineParserForDataToRetrieve implements TimelineParser {
+  private String expr;
+  private final int exprLength;
+  public TimelineParserForDataToRetrieve(String expression) {
+    this.expr = expression;
+    if (expression != null) {
+      this.expr = expr.trim();
+      exprLength = expr.length();
+    } else {
+      exprLength = 0;
+    }
+  }
+
+  @Override
+  public TimelineFilterList parse() throws TimelineParseException {
+    if (expr == null || exprLength == 0) {
+      return null;
+    }
+    TimelineCompareOp compareOp = null;
+    int openingBracketIndex =
+        expr.indexOf(TimelineParseConstants.OPENING_BRACKET_CHAR);
+    if (expr.charAt(0) == TimelineParseConstants.NOT_CHAR) {
+      if (openingBracketIndex == -1) {
+        throw new TimelineParseException("Invalid config/metric to retrieve " +
+            "expression");
+      }
+      if (openingBracketIndex != 1 &&
+          expr.substring(1, openingBracketIndex + 1).trim().length() != 1) {
+        throw new TimelineParseException("Invalid config/metric to retrieve " +
+            "expression");
+      }
+      compareOp = TimelineCompareOp.NOT_EQUAL;
+    } else if (openingBracketIndex <= 0) {
+      compareOp = TimelineCompareOp.EQUAL;
+    }
+    char lastChar = expr.charAt(exprLength - 1);
+    if (compareOp == TimelineCompareOp.NOT_EQUAL &&
+        lastChar != TimelineParseConstants.CLOSING_BRACKET_CHAR) {
+      throw new TimelineParseException("Invalid config/metric to retrieve " +
+          "expression");
+    }
+    if (openingBracketIndex != -1 &&
+        expr.charAt(exprLength - 1) ==
+            TimelineParseConstants.CLOSING_BRACKET_CHAR) {
+      expr = expr.substring(openingBracketIndex + 1, exprLength - 1).trim();
+    }
+    if (expr.isEmpty()) {
+      return null;
+    }
+    Operator op =
+        (compareOp == TimelineCompareOp.NOT_EQUAL) ? Operator.AND : Operator.OR;
+    TimelineFilterList list = new TimelineFilterList(op);
+    String[] splits = expr.split(TimelineParseConstants.COMMA_DELIMITER);
+    for (String split : splits) {
+      list.addFilter(new TimelinePrefixFilter(compareOp, split.trim()));
+    }
+    return list;
+  }
+
+  @Override
+  public void close() {
+  }
+}

+ 343 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForEqualityExpr.java

@@ -0,0 +1,343 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.timelineservice.reader;
+
+import java.util.Deque;
+import java.util.LinkedList;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList.Operator;
+
+/**
+ * Abstract class for parsing equality expressions. This means the values in
+ * expression would either be equal or not equal.
+ * Equality expressions are of the form :
+ * (&lt;value&gt;,&lt;value&gt;,&lt;value&gt;) &lt;op&gt; !(&lt;value&gt;,
+ * &lt;value&gt;)
+ *
+ * Here, "!" means all the values should not exist/should not be equal.
+ * If not specified, they should exist/be equal.
+ *
+ * op is a logical operator and can be either AND or OR.
+ *
+ * The way values will be interpreted would also depend on implementation.
+ *
+ * For instance for event filters this expression may look like,
+ * (event1,event2) AND !(event3,event4)
+ * This means for an entity to match, event1 and event2 should exist. But event3
+ * and event4 should not exist.
+ */
+@Private
+@Unstable
+abstract class TimelineParserForEqualityExpr implements TimelineParser {
+  private enum ParseState {
+    PARSING_VALUE,
+    PARSING_OP,
+    PARSING_COMPAREOP
+  }
+  private final String expr;
+  // Expression in lower case.
+  private final String exprInLowerCase;
+  // Expression name.
+  private final String exprName;
+  // Expression offset.
+  private int offset = 0;
+  // Offset used to parse values in the expression.
+  private int startOffset = 0;
+  private final int exprLength;
+  private ParseState currentParseState = ParseState.PARSING_COMPAREOP;
+  private TimelineCompareOp currentCompareOp = null;
+  // Used to store filter lists which can then be combined as brackets are
+  // closed.
+  private Deque<TimelineFilterList> filterListStack = new LinkedList<>();
+  private TimelineFilter currentFilter = null;
+  private TimelineFilterList filterList = null;
+  // Delimiter used to separate values.
+  private final char delimiter;
+  public TimelineParserForEqualityExpr(String expression, String name,
+      char delim) {
+    if (expression != null) {
+      expr = expression.trim();
+      exprLength = expr.length();
+      exprInLowerCase = expr.toLowerCase();
+    } else {
+      exprLength = 0;
+      expr = null;
+      exprInLowerCase = null;
+    }
+    exprName = name;
+    delimiter = delim;
+  }
+
+  protected TimelineFilter getCurrentFilter() {
+    return currentFilter;
+  }
+
+  protected TimelineFilter getFilterList() {
+    return filterList;
+  }
+
+  /**
+   * Creates filter as per implementation.
+   *
+   * @return a {@link TimelineFilter} implementation.
+   */
+  protected abstract TimelineFilter createFilter();
+
+  /**
+   * Sets compare op to the current filter as per filter implementation.
+   *
+   * @param compareOp compare op to be set.
+   * @throws Exception if any problem occurs.
+   */
+  protected abstract void setCompareOpToCurrentFilter(
+      TimelineCompareOp compareOp) throws TimelineParseException;
+
+  /**
+   * Sets value to the current filter as per filter implementation.
+   *
+   * @param value value to be set.
+   * @throws Exception if any problem occurs.
+   */
+  protected abstract void setValueToCurrentFilter(String value)
+      throws TimelineParseException;
+
+  private void createAndSetFilter(boolean checkIfNull)
+      throws TimelineParseException {
+    if (!checkIfNull || currentFilter == null) {
+      currentFilter = createFilter();
+      setCompareOpToCurrentFilter(currentCompareOp);
+    }
+    setValueToCurrentFilter(expr.substring(startOffset, offset).trim());
+  }
+
+  private void handleSpaceChar() throws TimelineParseException {
+    if (currentParseState == ParseState.PARSING_VALUE) {
+      if (startOffset == offset) {
+        startOffset++;
+      } else {
+        createAndSetFilter(true);
+        currentParseState = ParseState.PARSING_OP;
+      }
+    }
+    offset++;
+  }
+
+  private void handleDelimiter() throws TimelineParseException {
+    if (currentParseState == ParseState.PARSING_OP ||
+        currentParseState == ParseState.PARSING_VALUE) {
+      if (currentParseState == ParseState.PARSING_VALUE) {
+        createAndSetFilter(false);
+      }
+      if (filterList == null) {
+        filterList = new TimelineFilterList();
+      }
+      // Add parsed filter into filterlist and make it null to move on to next
+      // filter.
+      filterList.addFilter(currentFilter);
+      currentFilter = null;
+      offset++;
+      startOffset = offset;
+      currentParseState = ParseState.PARSING_VALUE;
+    } else {
+      throw new TimelineParseException("Invalid " + exprName + "expression.");
+    }
+  }
+
+  private void handleOpeningBracketChar(boolean encounteredNot)
+      throws TimelineParseException {
+    if (currentParseState == ParseState.PARSING_COMPAREOP ||
+        currentParseState == ParseState.PARSING_VALUE) {
+      offset++;
+      startOffset = offset;
+      filterListStack.push(filterList);
+      filterList = null;
+      if (currentFilter == null) {
+        currentFilter = createFilter();
+      }
+      currentCompareOp = encounteredNot ?
+          TimelineCompareOp.NOT_EQUAL : TimelineCompareOp.EQUAL;
+      setCompareOpToCurrentFilter(currentCompareOp);
+      currentParseState = ParseState.PARSING_VALUE;
+    } else {
+      throw new TimelineParseException("Encountered unexpected opening " +
+          "bracket while parsing " + exprName + ".");
+    }
+  }
+
+  private void handleNotChar() throws TimelineParseException {
+    if (currentParseState == ParseState.PARSING_COMPAREOP ||
+        currentParseState == ParseState.PARSING_VALUE) {
+      offset++;
+      while (offset < exprLength &&
+          expr.charAt(offset) == TimelineParseConstants.SPACE_CHAR) {
+        offset++;
+      }
+      if (offset == exprLength) {
+        throw new TimelineParseException("Invalid " + exprName + "expression");
+      }
+      if (expr.charAt(offset) == TimelineParseConstants.OPENING_BRACKET_CHAR) {
+        handleOpeningBracketChar(true);
+      } else {
+        throw new TimelineParseException("Invalid " + exprName + "expression");
+      }
+    } else {
+      throw new TimelineParseException("Encountered unexpected not(!) char " +
+         "while parsing " + exprName + ".");
+    }
+  }
+
+  private void handleClosingBracketChar() throws TimelineParseException {
+    if (currentParseState != ParseState.PARSING_VALUE &&
+        currentParseState != ParseState.PARSING_OP) {
+      throw new TimelineParseException("Encountered unexpected closing " +
+          "bracket while parsing " + exprName + ".");
+    }
+    if (!filterListStack.isEmpty()) {
+      if (currentParseState == ParseState.PARSING_VALUE) {
+        if (startOffset != offset) {
+          createAndSetFilter(true);
+          currentParseState = ParseState.PARSING_OP;
+        }
+      }
+      if (filterList == null) {
+        filterList = new TimelineFilterList();
+      }
+      if (currentFilter != null) {
+        filterList.addFilter(currentFilter);
+      }
+      // As bracket is closing, pop the filter list from top of the stack and
+      // combine it with current filter list.
+      TimelineFilterList fList = filterListStack.pop();
+      if (fList != null) {
+        fList.addFilter(filterList);
+        filterList = fList;
+      }
+      currentFilter = null;
+      offset++;
+      startOffset = offset;
+    } else {
+      throw new TimelineParseException("Encountered unexpected closing " +
+          "bracket while parsing " + exprName + ".");
+    }
+  }
+
+  private void parseOp(boolean closingBracket) throws TimelineParseException {
+    Operator operator = null;
+    if (exprInLowerCase.startsWith("or ", offset)) {
+      operator = Operator.OR;
+      offset = offset + 3;
+    } else if (exprInLowerCase.startsWith("and ", offset)) {
+      operator = Operator.AND;
+      offset = offset + 4;
+    }
+    if (operator == null) {
+      throw new TimelineParseException("Operator cannot be parsed for " +
+          exprName + ".");
+    }
+    if (filterList == null) {
+      filterList = new TimelineFilterList(operator);
+    }
+    if (currentFilter != null) {
+      filterList.addFilter(currentFilter);
+    }
+    if (closingBracket || filterList.getOperator() != operator) {
+      filterList = new TimelineFilterList(operator, filterList);
+    }
+    currentFilter = null;
+    startOffset = offset;
+    currentParseState = ParseState.PARSING_COMPAREOP;
+  }
+
+  private void parseCompareOp() throws TimelineParseException {
+    if (currentFilter == null) {
+      currentFilter = createFilter();
+    }
+    currentCompareOp = TimelineCompareOp.EQUAL;
+    setCompareOpToCurrentFilter(currentCompareOp);
+    currentParseState = ParseState.PARSING_VALUE;
+  }
+
+  @Override
+  public TimelineFilterList parse() throws TimelineParseException {
+    if (expr == null || exprLength == 0) {
+      return null;
+    }
+    boolean closingBracket = false;
+    while (offset < exprLength) {
+      char offsetChar = expr.charAt(offset);
+      switch(offsetChar) {
+      case TimelineParseConstants.NOT_CHAR:
+        handleNotChar();
+        break;
+      case TimelineParseConstants.SPACE_CHAR:
+        handleSpaceChar();
+        break;
+      case TimelineParseConstants.OPENING_BRACKET_CHAR:
+        handleOpeningBracketChar(false);
+        break;
+      case TimelineParseConstants.CLOSING_BRACKET_CHAR:
+        handleClosingBracketChar();
+        closingBracket = true;
+        break;
+      default: // other characters.
+        if (offsetChar == delimiter) {
+          handleDelimiter();
+        } else if (currentParseState == ParseState.PARSING_COMPAREOP) {
+          parseCompareOp();
+        } else if (currentParseState == ParseState.PARSING_OP) {
+          parseOp(closingBracket);
+          closingBracket = false;
+        } else {
+          offset++;
+        }
+        break;
+      }
+    }
+    if (!filterListStack.isEmpty()) {
+      filterListStack.clear();
+      throw new TimelineParseException("Encountered improper brackets while " +
+          "parsing " + exprName + ".");
+    }
+    if (currentParseState == ParseState.PARSING_VALUE) {
+      if (startOffset != offset) {
+        createAndSetFilter(true);
+      }
+    }
+    if (filterList == null || filterList.getFilterList().isEmpty()) {
+      filterList = new TimelineFilterList(currentFilter);
+    } else if (currentFilter != null) {
+      filterList.addFilter(currentFilter);
+    }
+    return filterList;
+  }
+
+  @Override
+  public void close() {
+    if (filterListStack != null) {
+      filterListStack.clear();
+    }
+    currentFilter = null;
+    filterList = null;
+  }
+}

+ 51 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForExistFilters.java

@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.timelineservice.reader;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter;
+
+/**
+ * Used for parsing existence filters such as event filters. These filters
+ * check for existence of a value. For example, in case of event filters, they
+ * check if an event exists or not and accordingly return an entity.
+ */
+@Private
+@Unstable
+class TimelineParserForExistFilters extends TimelineParserForEqualityExpr {
+
+  public TimelineParserForExistFilters(String expression, char delimiter) {
+    super(expression, "Event Filter", delimiter);
+  }
+
+  protected TimelineFilter createFilter() {
+    return new TimelineExistsFilter();
+  }
+
+  protected void setValueToCurrentFilter(String value) {
+    ((TimelineExistsFilter)getCurrentFilter()).setValue(value);
+  }
+
+  protected void setCompareOpToCurrentFilter(TimelineCompareOp compareOp) {
+    ((TimelineExistsFilter)getCurrentFilter()).setCompareOp(compareOp);
+  }
+}

+ 78 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForKVFilters.java

@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.timelineservice.reader;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter;
+
+/**
+ * Used for parsing key-value filters such as config and info filters.
+ */
+@Private
+@Unstable
+class TimelineParserForKVFilters extends TimelineParserForCompareExpr {
+  // Indicates if value has to be interpreted as a string.
+  private final boolean valueAsString;
+  public TimelineParserForKVFilters(String expression, boolean valAsStr) {
+    super(expression, "Config/Info Filter");
+    this.valueAsString = valAsStr;
+  }
+
+  protected TimelineFilter createFilter() {
+    return new TimelineKeyValueFilter();
+  }
+
+  protected Object parseValue(String strValue) {
+    if (!valueAsString) {
+      try {
+        return GenericObjectMapper.OBJECT_READER.readValue(strValue);
+      } catch (IOException e) {
+        return strValue;
+      }
+    } else {
+      return strValue;
+    }
+  }
+
+  @Override
+  protected void setCompareOpToCurrentFilter(TimelineCompareOp compareOp,
+      boolean keyMustExistFlag) throws TimelineParseException {
+    if (compareOp != TimelineCompareOp.EQUAL &&
+        compareOp != TimelineCompareOp.NOT_EQUAL) {
+      throw new TimelineParseException("TimelineCompareOp for kv-filter " +
+          "should be EQUAL or NOT_EQUAL");
+    }
+    ((TimelineKeyValueFilter)getCurrentFilter()).setCompareOp(
+        compareOp, keyMustExistFlag);
+  }
+
+  @Override
+  protected void setValueToCurrentFilter(Object value) {
+    TimelineFilter currentFilter = getCurrentFilter();
+    if (currentFilter != null) {
+      ((TimelineKeyValueFilter)currentFilter).setValue(value);
+    }
+  }
+}

+ 72 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForNumericFilters.java

@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.timelineservice.reader;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
+
+/**
+ * Used for parsing numerical filters such as metric filters.
+ */
+@Private
+@Unstable
+class TimelineParserForNumericFilters extends TimelineParserForCompareExpr {
+
+  public TimelineParserForNumericFilters(String expression) {
+    super(expression, "Metric Filter");
+  }
+
+  protected TimelineFilter createFilter() {
+    return new TimelineCompareFilter();
+  }
+
+  @Override
+  protected void setCompareOpToCurrentFilter(TimelineCompareOp compareOp,
+      boolean keyMustExistFlag) {
+    ((TimelineCompareFilter)getCurrentFilter()).setCompareOp(
+        compareOp, keyMustExistFlag);
+  }
+
+  protected Object parseValue(String strValue) throws TimelineParseException {
+    Object value = null;
+    try {
+      value = GenericObjectMapper.OBJECT_READER.readValue(strValue);
+    } catch (IOException e) {
+      throw new TimelineParseException("Value cannot be parsed.");
+    }
+    if (value == null || !(TimelineStorageUtils.isIntegralValue(value))) {
+      throw new TimelineParseException("Value is not a number.");
+    }
+    return value;
+  }
+
+  protected void setValueToCurrentFilter(Object value) {
+    TimelineFilter currentFilter = getCurrentFilter();
+    if (currentFilter != null) {
+      ((TimelineCompareFilter)currentFilter).setValue(value);
+    }
+  }
+}

+ 71 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForRelationFilters.java

@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.timelineservice.reader;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValuesFilter;
+
+/**
+ * Used for parsing relation filters.
+ */
+@Private
+@Unstable
+class TimelineParserForRelationFilters extends
+    TimelineParserForEqualityExpr {
+  private final String valueDelimiter;
+  public TimelineParserForRelationFilters(String expression, char valuesDelim,
+      String valueDelim) {
+    super(expression, "Relation Filter", valuesDelim);
+    valueDelimiter = valueDelim;
+  }
+
+  @Override
+  protected TimelineFilter createFilter() {
+    return new TimelineKeyValuesFilter();
+  }
+
+  @Override
+  protected void setCompareOpToCurrentFilter(TimelineCompareOp compareOp) {
+    ((TimelineKeyValuesFilter)getCurrentFilter()).setCompareOp(compareOp);
+  }
+
+  @Override
+  protected void setValueToCurrentFilter(String value)
+       throws TimelineParseException {
+    if (value != null) {
+      String[] pairStrs = value.split(valueDelimiter);
+      if (pairStrs.length < 2) {
+        throw new TimelineParseException("Invalid relation filter expression");
+      }
+      String key = pairStrs[0].trim();
+      Set<Object> values = new HashSet<Object>();
+      for (int i = 1; i < pairStrs.length; i++) {
+        values.add(pairStrs[i].trim());
+      }
+      ((TimelineKeyValuesFilter)getCurrentFilter()).
+          setKeyAndValues(key, values);
+    }
+  }
+}

+ 196 - 24
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java

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

+ 65 - 131
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java

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

+ 69 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineCompareFilter.java

@@ -29,12 +29,15 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 @Unstable
 @Unstable
 public class TimelineCompareFilter extends TimelineFilter {
 public class TimelineCompareFilter extends TimelineFilter {
 
 
-  private final TimelineCompareOp compareOp;
-  private final String key;
-  private final Object value;
+  private TimelineCompareOp compareOp;
+  private String key;
+  private Object value;
   // If comparison operator is NOT_EQUAL, this flag decides if we should return
   // If comparison operator is NOT_EQUAL, this flag decides if we should return
   // the entity if key does not exist.
   // the entity if key does not exist.
-  private final boolean keyMustExist;
+  private boolean keyMustExist = true;
+
+  public TimelineCompareFilter() {
+  }
 
 
   public TimelineCompareFilter(TimelineCompareOp op, String key, Object val,
   public TimelineCompareFilter(TimelineCompareOp op, String key, Object val,
        boolean keyMustExistFlag) {
        boolean keyMustExistFlag) {
@@ -65,14 +68,76 @@ public class TimelineCompareFilter extends TimelineFilter {
     return key;
     return key;
   }
   }
 
 
+  public void setKey(String keyToBeSet) {
+    key = keyToBeSet;
+  }
+
   public Object getValue() {
   public Object getValue() {
     return value;
     return value;
   }
   }
 
 
+  public void setCompareOp(TimelineCompareOp timelineCompareOp,
+      boolean keyExistFlag) {
+    this.compareOp = timelineCompareOp;
+    if (timelineCompareOp == TimelineCompareOp.NOT_EQUAL) {
+      this.keyMustExist = keyExistFlag;
+    }
+  }
+
+  public void setValue(Object val) {
+    value = val;
+  }
+
   public boolean getKeyMustExist() {
   public boolean getKeyMustExist() {
     return keyMustExist;
     return keyMustExist;
   }
   }
 
 
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((compareOp == null) ? 0 : compareOp.hashCode());
+    result = prime * result + ((key == null) ? 0 : key.hashCode());
+    result = prime * result + (keyMustExist ? 1231 : 1237);
+    result = prime * result + ((value == null) ? 0 : value.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    TimelineCompareFilter other = (TimelineCompareFilter) obj;
+    if (compareOp != other.compareOp) {
+      return false;
+    }
+    if (key == null) {
+      if (other.key != null) {
+        return false;
+      }
+    } else if (!key.equals(other.key)) {
+      return false;
+    }
+    if (keyMustExist != other.keyMustExist) {
+      return false;
+    }
+    if (value == null) {
+      if (other.value != null) {
+        return false;
+      }
+    } else if (!value.equals(other.value)) {
+      return false;
+    }
+    return true;
+  }
+
   @Override
   @Override
   public String toString() {
   public String toString() {
     return String.format("%s (%s, %s:%s:%b)",
     return String.format("%s (%s, %s:%s:%b)",

+ 47 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineExistsFilter.java

@@ -29,8 +29,11 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 @Unstable
 @Unstable
 public class TimelineExistsFilter extends TimelineFilter {
 public class TimelineExistsFilter extends TimelineFilter {
 
 
-  private final TimelineCompareOp compareOp;
-  private final String value;
+  private TimelineCompareOp compareOp;
+  private String value;
+
+  public TimelineExistsFilter() {
+  }
 
 
   public TimelineExistsFilter(TimelineCompareOp op, String value) {
   public TimelineExistsFilter(TimelineCompareOp op, String value) {
     this.value = value;
     this.value = value;
@@ -41,15 +44,57 @@ public class TimelineExistsFilter extends TimelineFilter {
     this.compareOp = op;
     this.compareOp = op;
   }
   }
 
 
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((compareOp == null) ? 0 : compareOp.hashCode());
+    result = prime * result + ((value == null) ? 0 : value.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    TimelineExistsFilter other = (TimelineExistsFilter) obj;
+    if (compareOp != other.compareOp) {
+      return false;
+    }
+    if (value == null) {
+      if (other.value != null) {
+        return false;
+      }
+    } else if (!value.equals(other.value)) {
+      return false;
+    }
+    return true;
+  }
+
   @Override
   @Override
   public TimelineFilterType getFilterType() {
   public TimelineFilterType getFilterType() {
     return TimelineFilterType.EXISTS;
     return TimelineFilterType.EXISTS;
   }
   }
 
 
+  public void setValue(String val) {
+    value = val;
+  }
+
   public String getValue() {
   public String getValue() {
     return value;
     return value;
   }
   }
 
 
+  public void setCompareOp(TimelineCompareOp op) {
+    compareOp = op;
+  }
+
   public TimelineCompareOp getCompareOp() {
   public TimelineCompareOp getCompareOp() {
     return compareOp;
     return compareOp;
   }
   }

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

@@ -97,6 +97,42 @@ public class TimelineFilterList extends TimelineFilter {
     filterList.add(filter);
     filterList.add(filter);
   }
   }
 
 
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result =
+        prime * result + ((filterList == null) ? 0 : filterList.hashCode());
+    result =
+        prime * result + ((operator == null) ? 0 : operator.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    TimelineFilterList other = (TimelineFilterList) obj;
+    if (operator != other.operator) {
+      return false;
+    }
+    if (filterList == null) {
+      if (other.filterList != null) {
+        return false;
+      }
+    } else if (!filterList.equals(other.filterList)) {
+      return false;
+    }
+    return true;
+  }
+
   @Override
   @Override
   public String toString() {
   public String toString() {
     return String.format("TimelineFilterList %s (%d): %s",
     return String.format("TimelineFilterList %s (%d): %s",

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

@@ -28,6 +28,9 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 @Private
 @Private
 @Unstable
 @Unstable
 public class TimelineKeyValueFilter extends TimelineCompareFilter {
 public class TimelineKeyValueFilter extends TimelineCompareFilter {
+  public TimelineKeyValueFilter() {
+  }
+
   public TimelineKeyValueFilter(TimelineCompareOp op, String key, Object val,
   public TimelineKeyValueFilter(TimelineCompareOp op, String key, Object val,
       boolean keyMustExistFlag) {
       boolean keyMustExistFlag) {
     super(op, key, val, keyMustExistFlag);
     super(op, key, val, keyMustExistFlag);
@@ -45,4 +48,14 @@ public class TimelineKeyValueFilter extends TimelineCompareFilter {
   public TimelineFilterType getFilterType() {
   public TimelineFilterType getFilterType() {
     return TimelineFilterType.KEY_VALUE;
     return TimelineFilterType.KEY_VALUE;
   }
   }
+
+  public void setCompareOp(TimelineCompareOp timelineCompareOp,
+      boolean keyExistFlag) {
+    if (timelineCompareOp != TimelineCompareOp.EQUAL &&
+        timelineCompareOp != TimelineCompareOp.NOT_EQUAL) {
+      throw new IllegalArgumentException("TimelineCompareOp for equality"
+          + " filter should be EQUAL or NOT_EQUAL");
+    }
+    super.setCompareOp(timelineCompareOp, keyExistFlag);
+  }
 }
 }

+ 58 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValuesFilter.java

@@ -31,9 +31,13 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 @Private
 @Private
 @Unstable
 @Unstable
 public class TimelineKeyValuesFilter extends TimelineFilter {
 public class TimelineKeyValuesFilter extends TimelineFilter {
-  private final TimelineCompareOp compareOp;
-  private final String key;
-  private final Set<Object> values;
+  private TimelineCompareOp compareOp;
+  private String key;
+  private Set<Object> values;
+
+  public TimelineKeyValuesFilter() {
+  }
+
   public TimelineKeyValuesFilter(TimelineCompareOp op, String key,
   public TimelineKeyValuesFilter(TimelineCompareOp op, String key,
       Set<Object> values) {
       Set<Object> values) {
     if (op != TimelineCompareOp.EQUAL && op != TimelineCompareOp.NOT_EQUAL) {
     if (op != TimelineCompareOp.EQUAL && op != TimelineCompareOp.NOT_EQUAL) {
@@ -58,6 +62,15 @@ public class TimelineKeyValuesFilter extends TimelineFilter {
     return values;
     return values;
   }
   }
 
 
+  public void setKeyAndValues(String keyForValues, Set<Object> vals) {
+    key = keyForValues;
+    values = vals;
+  }
+
+  public void setCompareOp(TimelineCompareOp op) {
+    compareOp = op;
+  }
+
   public TimelineCompareOp getCompareOp() {
   public TimelineCompareOp getCompareOp() {
     return compareOp;
     return compareOp;
   }
   }
@@ -68,4 +81,46 @@ public class TimelineKeyValuesFilter extends TimelineFilter {
         this.getClass().getSimpleName(), this.compareOp.name(),
         this.getClass().getSimpleName(), this.compareOp.name(),
         this.key, (values == null) ? "" : values.toString());
         this.key, (values == null) ? "" : values.toString());
   }
   }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((compareOp == null) ? 0 : compareOp.hashCode());
+    result = prime * result + ((key == null) ? 0 : key.hashCode());
+    result = prime * result + ((values == null) ? 0 : values.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    TimelineKeyValuesFilter other = (TimelineKeyValuesFilter) obj;
+    if (compareOp != other.compareOp) {
+      return false;
+    }
+    if (key == null) {
+      if (other.key != null) {
+        return false;
+      }
+    } else if (!key.equals(other.key)) {
+      return false;
+    }
+    if (values == null) {
+      if (other.values != null) {
+        return false;
+      }
+    } else if (!values.equals(other.values)) {
+      return false;
+    }
+    return true;
+  }
 }
 }

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

@@ -32,6 +32,9 @@ public class TimelinePrefixFilter extends TimelineFilter {
   private TimelineCompareOp compareOp;
   private TimelineCompareOp compareOp;
   private String prefix;
   private String prefix;
 
 
+  public TimelinePrefixFilter() {
+  }
+
   public TimelinePrefixFilter(TimelineCompareOp op, String prefix) {
   public TimelinePrefixFilter(TimelineCompareOp op, String prefix) {
     this.prefix = prefix;
     this.prefix = prefix;
     if (op != TimelineCompareOp.EQUAL && op != TimelineCompareOp.NOT_EQUAL) {
     if (op != TimelineCompareOp.EQUAL && op != TimelineCompareOp.NOT_EQUAL) {
@@ -59,4 +62,38 @@ public class TimelinePrefixFilter extends TimelineFilter {
     return String.format("%s (%s %s)",
     return String.format("%s (%s %s)",
         this.getClass().getSimpleName(), this.compareOp.name(), this.prefix);
         this.getClass().getSimpleName(), this.compareOp.name(), this.prefix);
   }
   }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((compareOp == null) ? 0 : compareOp.hashCode());
+    result = prime * result + ((prefix == null) ? 0 : prefix.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    TimelinePrefixFilter other = (TimelinePrefixFilter) obj;
+    if (compareOp != other.compareOp) {
+      return false;
+    }
+    if (prefix == null) {
+      if (other.prefix != null) {
+        return false;
+      }
+    } else if (!prefix.equals(other.prefix)){
+      return false;
+    }
+    return true;
+  }
 }
 }

+ 7 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java

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

File diff suppressed because it is too large
+ 869 - 21
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java


+ 923 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesUtils.java

@@ -0,0 +1,923 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.timelineservice.reader;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList.Operator;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValuesFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter;
+import org.junit.Test;
+
+import com.google.common.collect.Sets;
+
+public class TestTimelineReaderWebServicesUtils {
+  private static void verifyFilterList(String expr, TimelineFilterList list,
+      TimelineFilterList expectedList) throws Exception {
+    assertNotNull(list);
+    assertTrue("Unexpected List received after parsing expression " + expr +
+        ". Expected=" + expectedList + " but Actual=" + list,
+        list.equals(expectedList));
+  }
+
+  @Test
+  public void testMetricFiltersParsing() throws Exception {
+    String expr = "(((key11 ne 234 AND key12 gt 23) AND " +
+        "(key13 lt 34 OR key14 ge 567)) OR (key21 lt 24 OR key22 le 45))";
+    TimelineFilterList expectedList = new TimelineFilterList(
+        Operator.OR,
+        new TimelineFilterList(
+            Operator.AND,
+            new TimelineFilterList(
+                Operator.AND,
+                new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL,
+                    "key11", 234, false),
+                new TimelineCompareFilter(TimelineCompareOp.GREATER_THAN,
+                    "key12", 23, true)
+            ),
+            new TimelineFilterList(
+                Operator.OR,
+                new TimelineCompareFilter(TimelineCompareOp.LESS_THAN,
+                    "key13", 34, true),
+                new TimelineCompareFilter(TimelineCompareOp.GREATER_OR_EQUAL,
+                    "key14", 567, true)
+            )
+        ),
+        new TimelineFilterList(
+            Operator.OR,
+            new TimelineCompareFilter(TimelineCompareOp.LESS_THAN,
+                "key21", 24, true),
+            new TimelineCompareFilter(TimelineCompareOp.LESS_OR_EQUAL,
+                "key22", 45, true)
+        )
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList);
+
+    expr = "abc ene 234";
+    expectedList = new TimelineFilterList(
+        new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL,
+            "abc", 234, true)
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList);
+
+    expr = "abc ne 234";
+    expectedList = new TimelineFilterList(
+        new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL,
+            "abc", 234, false)
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList);
+
+    expr = "abc ne 234 AND def gt 23";
+    expectedList = new TimelineFilterList(
+        new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL,
+            "abc", 234, false),
+        new TimelineCompareFilter(TimelineCompareOp.GREATER_THAN,
+            "def", 23, true)
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList);
+
+    expr = "(abc ne 234 AND def gt 23)";
+    expectedList = new TimelineFilterList(
+        new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL,
+            "abc", 234, false),
+        new TimelineCompareFilter(TimelineCompareOp.GREATER_THAN,
+            "def", 23, true)
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList);
+
+    expr = "abc ne 234 AND def gt 23 OR rst lt 24";
+    expectedList = new TimelineFilterList(
+        Operator.OR,
+        new TimelineFilterList(
+            new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL,
+                "abc", 234, false),
+            new TimelineCompareFilter(TimelineCompareOp.GREATER_THAN,
+                "def", 23, true)
+        ),
+        new TimelineCompareFilter(TimelineCompareOp.LESS_THAN,
+            "rst", 24, true)
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList);
+
+    expr = "abc ne 234 AND def gt 23 OR rst lt 24 OR xyz le 456";
+    expectedList = new TimelineFilterList(
+        Operator.OR,
+        new TimelineFilterList(
+            new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL,
+                "abc", 234, false),
+            new TimelineCompareFilter(TimelineCompareOp.GREATER_THAN,
+                "def", 23, true)
+        ),
+        new TimelineCompareFilter(TimelineCompareOp.LESS_THAN,
+            "rst", 24, true),
+        new TimelineCompareFilter(TimelineCompareOp.LESS_OR_EQUAL,
+            "xyz", 456, true)
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList);
+
+    expr = "abc ne 234 AND def gt 23 OR rst lt 24 OR xyz le 456 AND pqr ge 2";
+    expectedList = new TimelineFilterList(
+        new TimelineFilterList(
+            Operator.OR,
+            new TimelineFilterList(
+                new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL,
+                    "abc", 234, false),
+                new TimelineCompareFilter(TimelineCompareOp.GREATER_THAN,
+                    "def", 23, true)
+            ),
+            new TimelineCompareFilter(TimelineCompareOp.LESS_THAN,
+                "rst", 24, true),
+            new TimelineCompareFilter(TimelineCompareOp.LESS_OR_EQUAL,
+                "xyz", 456, true)
+        ),
+        new TimelineCompareFilter(TimelineCompareOp.GREATER_OR_EQUAL,
+            "pqr", 2, true)
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList);
+
+    // Test with unnecessary spaces.
+    expr = "  abc ne   234       AND       def           gt 23 OR     rst lt "+
+        "           24     OR xyz     le     456    AND pqr ge 2        ";
+    expectedList = new TimelineFilterList(
+        new TimelineFilterList(
+            Operator.OR,
+            new TimelineFilterList(
+                new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL,
+                    "abc", 234, false),
+                new TimelineCompareFilter(TimelineCompareOp.GREATER_THAN,
+                    "def", 23, true)
+            ),
+            new TimelineCompareFilter(TimelineCompareOp.LESS_THAN,
+                "rst", 24, true),
+            new TimelineCompareFilter(TimelineCompareOp.LESS_OR_EQUAL,
+                "xyz", 456, true)
+        ),
+        new TimelineCompareFilter(TimelineCompareOp.GREATER_OR_EQUAL,
+            "pqr", 2, true)
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList);
+
+    expr = "(((key11 ne 234 AND key12 gt 23 OR key13 lt 24 OR key14 le 456 " +
+        "AND key15 ge 2) AND (key16 lt 34 OR key17 ge 567)) OR (key21 lt 24 " +
+        "OR key22 le 45))";
+    expectedList = new TimelineFilterList(
+        Operator.OR,
+        new TimelineFilterList(
+            Operator.AND,
+            new TimelineFilterList(
+                new TimelineFilterList(
+                    Operator.OR,
+                    new TimelineFilterList(
+                        new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL,
+                            "key11", 234, false),
+                        new TimelineCompareFilter(
+                            TimelineCompareOp.GREATER_THAN, "key12", 23, true)
+                    ),
+                    new TimelineCompareFilter(TimelineCompareOp.LESS_THAN,
+                        "key13", 24, true),
+                    new TimelineCompareFilter(TimelineCompareOp.LESS_OR_EQUAL,
+                        "key14", 456, true)
+                ),
+                new TimelineCompareFilter(TimelineCompareOp.GREATER_OR_EQUAL,
+                    "key15", 2, true)
+            ),
+            new TimelineFilterList(
+                Operator.OR,
+                new TimelineCompareFilter(TimelineCompareOp.LESS_THAN,
+                    "key16", 34, true),
+                new TimelineCompareFilter(TimelineCompareOp.GREATER_OR_EQUAL,
+                    "key17", 567, true)
+            )
+        ),
+        new TimelineFilterList(
+            Operator.OR,
+            new TimelineCompareFilter(TimelineCompareOp.LESS_THAN,
+                "key21", 24, true),
+            new TimelineCompareFilter(TimelineCompareOp.LESS_OR_EQUAL,
+                "key22", 45, true)
+        )
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList);
+
+    expr = "   (  (     (        key11      ne     234    AND key12    gt   " +
+        "23    OR    key13    lt    24 OR key14 le 456   AND   key15   ge   2" +
+        "   )   AND ( key16 lt 34 OR key17 ge 567 )    )     OR " +
+        "(   key21 lt 24 OR key22 le 45 )   )    ";
+    expectedList = new TimelineFilterList(
+        Operator.OR,
+        new TimelineFilterList(
+            Operator.AND,
+            new TimelineFilterList(
+                new TimelineFilterList(
+                    Operator.OR,
+                    new TimelineFilterList(
+                        new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL,
+                            "key11", 234, false),
+                        new TimelineCompareFilter(
+                            TimelineCompareOp.GREATER_THAN, "key12", 23, true)
+                    ),
+                    new TimelineCompareFilter(TimelineCompareOp.LESS_THAN,
+                        "key13", 24, true),
+                    new TimelineCompareFilter(TimelineCompareOp.LESS_OR_EQUAL,
+                        "key14", 456, true)
+                ),
+                new TimelineCompareFilter(TimelineCompareOp.GREATER_OR_EQUAL,
+                    "key15", 2, true)
+            ),
+            new TimelineFilterList(
+                Operator.OR,
+                new TimelineCompareFilter(TimelineCompareOp.LESS_THAN,
+                    "key16", 34, true),
+                new TimelineCompareFilter(TimelineCompareOp.GREATER_OR_EQUAL,
+                    "key17", 567, true)
+            )
+        ),
+        new TimelineFilterList(
+            Operator.OR,
+            new TimelineCompareFilter(TimelineCompareOp.LESS_THAN,
+                "key21", 24, true),
+            new TimelineCompareFilter(TimelineCompareOp.LESS_OR_EQUAL,
+                "key22", 45, true)
+        )
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList);
+
+    expr = "(((key11 ne 234 AND key12 gt 23 OR key13 lt 24 OR key14 le 456 " +
+        "AND key15 ge 2) AND (key16 lt 34 OR key17 ge 567)) OR (key21 lt 24 " +
+        "OR key22 le 45)";
+    try {
+      TimelineReaderWebServicesUtils.parseMetricFilters(expr);
+      fail("Improper brackers. Exception should have been thrown.");
+    } catch (TimelineParseException e) {}
+
+    expr = "(((key11 ne 234 AND key12 gt v3 OR key13 lt 24 OR key14 le 456 " +
+        "AND key15 ge 2) AND (key16 lt 34 OR key17 ge 567)) OR (key21 lt 24 " +
+        "OR key22 le 45))";
+    try {
+      TimelineReaderWebServicesUtils.parseMetricFilters(expr);
+      fail("Non Numeric value. Exception should have been thrown.");
+    } catch (TimelineParseException e) {}
+
+    expr = "(((key11 ne (234 AND key12 gt 3 OR key13 lt 24 OR key14 le 456 " +
+        "AND key15 ge 2) AND (key16 lt 34 OR key17 ge 567)) OR (key21 lt 24 " +
+        "OR key22 le 45))";
+    try {
+      TimelineReaderWebServicesUtils.parseMetricFilters(expr);
+      fail("Unexpected opening bracket. Exception should have been thrown.");
+    } catch (TimelineParseException e) {}
+
+    expr = "(((k)ey11 ne 234 AND key12 gt 3 OR key13 lt 24 OR key14 le 456 " +
+        "AND key15 ge 2) AND (key16 lt 34 OR key17 ge 567)) OR (key21 lt 24 " +
+        "OR key22 le 45))";
+    try {
+      TimelineReaderWebServicesUtils.parseMetricFilters(expr);
+      fail("Unexpected closing bracket. Exception should have been thrown.");
+    } catch (TimelineParseException e) {}
+
+    expr = "(((key11 rs 234 AND key12 gt 3 OR key13 lt 24 OR key14 le 456 " +
+        "AND key15 ge 2) AND (key16 lt 34 OR key17 ge 567)) OR (key21 lt 24 " +
+        "OR key22 le 45))";
+    try {
+      TimelineReaderWebServicesUtils.parseMetricFilters(expr);
+      fail("Improper compare op. Exception should have been thrown.");
+    } catch (TimelineParseException e) {}
+
+    expr = "(((key11 ne 234 PI key12 gt 3 OR key13 lt 24 OR key14 le 456 " +
+        "AND key15 ge 2) AND (key16 lt 34 OR key17 ge 567)) OR (key21 lt 24 " +
+        "OR key22 le 45))";
+    try {
+      TimelineReaderWebServicesUtils.parseMetricFilters(expr);
+      fail("Improper op. Exception should have been thrown.");
+    } catch (TimelineParseException e) {}
+
+    expr = "(((key11 ne 234 PI key12 gt 3 OR key13 lt 24 OR key14 le 456 " +
+        "AND key15 ge 2) AND (key16 lt 34 OR key17 ge 567)) OR (key21 lt 24 " +
+        "OR key22 le 45))";
+    try {
+      TimelineReaderWebServicesUtils.parseMetricFilters(expr);
+      fail("Improper op. Exception should have been thrown.");
+    } catch (TimelineParseException e) {}
+
+    expr = "(key11 ne 234 AND key12 gt 3)) OR (key13 lt 24 OR key14 le 456)";
+    try {
+      TimelineReaderWebServicesUtils.parseMetricFilters(expr);
+      fail("Unbalanced brackets. Exception should have been thrown.");
+    } catch (TimelineParseException e) {}
+
+    expr = "(key11 rne 234 AND key12 gt 3) OR (key13 lt 24 OR key14 le 456)";
+    try {
+      TimelineReaderWebServicesUtils.parseMetricFilters(expr);
+      fail("Invalid compareop. Exception should have been thrown.");
+    } catch (TimelineParseException e) {}
+
+    expr = "(key11 ne 234 AND key12 gt 3) OR (key13 lt 24 OR key14 le";
+    try {
+      TimelineReaderWebServicesUtils.parseMetricFilters(expr);
+      fail("Compareop cant be parsed. Exception should have been thrown.");
+    } catch (TimelineParseException e) {}
+
+    assertNull(TimelineReaderWebServicesUtils.parseMetricFilters(null));
+    assertNull(TimelineReaderWebServicesUtils.parseMetricFilters("   "));
+  }
+
+  @Test
+  public void testConfigFiltersParsing() throws Exception {
+    String expr = "(((key11 ne 234 AND key12 eq val12) AND " +
+        "(key13 ene val13 OR key14 eq 567)) OR (key21 eq val_21 OR key22 eq " +
+        "val.22))";
+    TimelineFilterList expectedList = new TimelineFilterList(
+        Operator.OR,
+        new TimelineFilterList(
+            Operator.AND,
+            new TimelineFilterList(
+                Operator.AND,
+                new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL,
+                    "key11", "234", false),
+                new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                    "key12", "val12", true)
+            ),
+            new TimelineFilterList(
+                Operator.OR,
+                new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL,
+                    "key13", "val13", true),
+                new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                    "key14", "567", true)
+            )
+        ),
+        new TimelineFilterList(
+            Operator.OR,
+            new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                "key21", "val_21", true),
+            new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                "key22", "val.22", true)
+        )
+    );
+    verifyFilterList(expr, TimelineReaderWebServicesUtils.
+        parseKVFilters(expr, true), expectedList);
+
+    expr = "abc ne 234 AND def eq 23 OR rst ene 24 OR xyz eq 456 AND pqr eq 2";
+    expectedList = new TimelineFilterList(
+        new TimelineFilterList(
+            Operator.OR,
+            new TimelineFilterList(
+                new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL,
+                    "abc", "234", false),
+                new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                    "def", "23", true)
+            ),
+            new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL,
+                "rst", "24", true),
+            new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                "xyz", "456", true)
+        ),
+        new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+            "pqr", "2", true)
+    );
+    verifyFilterList(expr, TimelineReaderWebServicesUtils.
+        parseKVFilters(expr, true), expectedList);
+
+    // Test with unnecessary spaces.
+    expr = "  abc ne   234       AND       def           eq 23 OR     rst ene "+
+        "           24     OR xyz     eq     456    AND pqr eq 2        ";
+    expectedList = new TimelineFilterList(
+        new TimelineFilterList(
+            Operator.OR,
+            new TimelineFilterList(
+                new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL,
+                    "abc", "234", false),
+                new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                    "def", "23", true)
+            ),
+            new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL,
+                "rst", "24", true),
+            new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                "xyz", "456", true)
+        ),
+        new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+            "pqr", "2", true)
+    );
+    verifyFilterList(expr, TimelineReaderWebServicesUtils.
+        parseKVFilters(expr, true), expectedList);
+
+    expr = "abc gt 234 AND def eq 23 OR rst ene 24 OR xyz eq 456 AND pqr eq 2";
+    try {
+      TimelineReaderWebServicesUtils.parseKVFilters(expr, true);
+      fail("Invalid compareop specified for config filters. Should be either" +
+          " eq,ne or ene and exception should have been thrown.");
+    } catch (TimelineParseException e){}
+  }
+  @Test
+  public void testInfoFiltersParsing() throws Exception {
+    String expr = "(((key11 ne 234 AND key12 eq val12) AND " +
+        "(key13 ene val13 OR key14 eq 567)) OR (key21 eq val_21 OR key22 eq " +
+        "5.0))";
+    TimelineFilterList expectedList = new TimelineFilterList(
+        Operator.OR,
+        new TimelineFilterList(
+            Operator.AND,
+            new TimelineFilterList(
+                Operator.AND,
+                new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL,
+                    "key11", 234, false),
+                new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                    "key12", "val12", true)
+            ),
+            new TimelineFilterList(
+                Operator.OR,
+                new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL,
+                    "key13", "val13", true),
+                new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                    "key14", 567, true)
+            )
+        ),
+        new TimelineFilterList(
+            Operator.OR,
+            new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                "key21", "val_21", true),
+            new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                "key22", 5.0, true)
+        )
+    );
+    verifyFilterList(expr, TimelineReaderWebServicesUtils.
+        parseKVFilters(expr, false), expectedList);
+
+    expr = "abc ne 234 AND def eq 23 OR rst ene 24 OR xyz eq 456 AND pqr eq " +
+        "val.1234";
+    expectedList = new TimelineFilterList(
+        new TimelineFilterList(
+            Operator.OR,
+            new TimelineFilterList(
+                new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL,
+                    "abc", 234, false),
+                new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                    "def", 23, true)
+            ),
+            new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL,
+                "rst", 24, true),
+            new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                "xyz", 456, true)
+        ),
+        new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+            "pqr", "val.1234", true)
+    );
+    verifyFilterList(expr, TimelineReaderWebServicesUtils.
+        parseKVFilters(expr, false), expectedList);
+
+    // Test with unnecessary spaces.
+    expr = "  abc ne   234       AND       def           eq 23 OR     rst ene "+
+        "           24     OR xyz     eq     456    AND pqr eq 2        ";
+    expectedList = new TimelineFilterList(
+        new TimelineFilterList(
+            Operator.OR,
+            new TimelineFilterList(
+                new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL,
+                    "abc", 234, false),
+                new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                    "def", 23, true)
+            ),
+            new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL,
+                "rst", 24, true),
+            new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                "xyz", 456, true)
+        ),
+        new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+            "pqr", 2, true)
+    );
+    verifyFilterList(expr, TimelineReaderWebServicesUtils.
+        parseKVFilters(expr, false), expectedList);
+  }
+
+  @Test
+  public void testEventFiltersParsing() throws Exception {
+    String expr = "abc,def";
+    TimelineFilterList expectedList = new TimelineFilterList(
+        new TimelineExistsFilter(TimelineCompareOp.EQUAL, "abc"),
+        new TimelineExistsFilter(TimelineCompareOp.EQUAL, "def")
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseEventFilters(expr), expectedList);
+
+    expr = "(abc,def)";
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseEventFilters(expr), expectedList);
+
+    expr = "(abc,def) OR (rst, uvx)";
+    expectedList = new TimelineFilterList(Operator.OR,
+        new TimelineFilterList(
+            new TimelineExistsFilter(TimelineCompareOp.EQUAL, "abc"),
+            new TimelineExistsFilter(TimelineCompareOp.EQUAL, "def")
+        ),
+        new TimelineFilterList(
+            new TimelineExistsFilter(TimelineCompareOp.EQUAL, "rst"),
+            new TimelineExistsFilter(TimelineCompareOp.EQUAL, "uvx")
+        )
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseEventFilters(expr), expectedList);
+
+    expr = "!(abc,def,uvc) OR (rst, uvx)";
+    expectedList = new TimelineFilterList(Operator.OR,
+        new TimelineFilterList(
+            new TimelineExistsFilter(TimelineCompareOp.NOT_EQUAL, "abc"),
+            new TimelineExistsFilter(TimelineCompareOp.NOT_EQUAL, "def"),
+            new TimelineExistsFilter(TimelineCompareOp.NOT_EQUAL, "uvc")
+        ),
+        new TimelineFilterList(
+            new TimelineExistsFilter(TimelineCompareOp.EQUAL, "rst"),
+            new TimelineExistsFilter(TimelineCompareOp.EQUAL, "uvx")
+        )
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseEventFilters(expr), expectedList);
+
+    expr = "(((!(abc,def,uvc) OR (rst, uvx)) AND (!(abcdefg) OR !(ghj,tyu)))" +
+        " OR ((bcd,tyu) AND uvb))";
+    expectedList = new TimelineFilterList(Operator.OR,
+        new TimelineFilterList(
+            new TimelineFilterList(Operator.OR,
+                new TimelineFilterList(
+                    new TimelineExistsFilter(
+                        TimelineCompareOp.NOT_EQUAL, "abc"),
+                    new TimelineExistsFilter(
+                        TimelineCompareOp.NOT_EQUAL, "def"),
+                    new TimelineExistsFilter(TimelineCompareOp.NOT_EQUAL, "uvc")
+                ),
+                new TimelineFilterList(
+                    new TimelineExistsFilter(TimelineCompareOp.EQUAL, "rst"),
+                    new TimelineExistsFilter(TimelineCompareOp.EQUAL, "uvx")
+                )
+            ),
+            new TimelineFilterList(Operator.OR,
+                new TimelineFilterList(
+                    new TimelineExistsFilter(
+                        TimelineCompareOp.NOT_EQUAL, "abcdefg")
+                ),
+                new TimelineFilterList(
+                    new TimelineExistsFilter(
+                        TimelineCompareOp.NOT_EQUAL, "ghj"),
+                    new TimelineExistsFilter(
+                        TimelineCompareOp.NOT_EQUAL, "tyu")
+                )
+            )
+        ),
+        new TimelineFilterList(
+            new TimelineFilterList(
+                new TimelineExistsFilter(TimelineCompareOp.EQUAL, "bcd"),
+                new TimelineExistsFilter(TimelineCompareOp.EQUAL, "tyu")
+            ),
+            new TimelineExistsFilter(TimelineCompareOp.EQUAL, "uvb")
+        )
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseEventFilters(expr), expectedList);
+
+    expr = "  (  (  (  !  (  abc , def  ,   uvc)   OR   (   rst  ,   uvx )  )" +
+        "  AND   (  !  (  abcdefg ) OR  !   (  ghj,  tyu)  ) )  OR   (   (" +
+        "   bcd   ,   tyu  )   AND   uvb  )   )";
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseEventFilters(expr), expectedList);
+
+    expr = "(((!(abc,def,uvc) OR (rst, uvx)) AND (!(abcdefg) OR !(ghj,tyu)))" +
+        " OR ((bcd,tyu) AND uvb)";
+    try {
+      TimelineReaderWebServicesUtils.parseEventFilters(expr);
+      fail("Improper brackets. Exception should have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "(((!(abc,def,uvc) (OR (rst, uvx)) AND (!(abcdefg) OR !(ghj,tyu)))" +
+        " OR ((bcd,tyu) AND uvb))";
+    try {
+      TimelineReaderWebServicesUtils.parseEventFilters(expr);
+      fail("Unexpected opening bracket. Exception should have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "(((!(abc,def,uvc) OR) (rst, uvx)) AND (!(abcdefg) OR !(ghj,tyu)))" +
+        " OR ((bcd,tyu) AND uvb))";
+    try {
+      TimelineReaderWebServicesUtils.parseEventFilters(expr);
+      fail("Unexpected closing bracket. Exception should have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "(((!(abc,def,uvc) PI (rst, uvx)) AND (!(abcdefg) OR !(ghj,tyu)))" +
+        " OR ((bcd,tyu) AND uvb))";
+    try {
+      TimelineReaderWebServicesUtils.parseEventFilters(expr);
+      fail("Invalid op. Exception should have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "(((!(abc,def,uvc) !OR (rst, uvx)) AND (!(abcdefg) OR !(ghj,tyu)))" +
+        " OR ((bcd,tyu) AND uvb))";
+    try {
+      TimelineReaderWebServicesUtils.parseEventFilters(expr);
+      fail("Unexpected ! char. Exception should have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "abc,def,uvc) OR (rst, uvx)";
+    try {
+      TimelineReaderWebServicesUtils.parseEventFilters(expr);
+      fail("Unexpected closing bracket. Exception should have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "abc,def,uvc OR )rst, uvx)";
+    try {
+      TimelineReaderWebServicesUtils.parseEventFilters(expr);
+      fail("Unexpected closing bracket. Exception should have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "abc,def,uvc OR ,rst, uvx)";
+    try {
+      TimelineReaderWebServicesUtils.parseEventFilters(expr);
+      fail("Unexpected delimiter. Exception should have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "abc,def,uvc OR !  ";
+    try {
+      TimelineReaderWebServicesUtils.parseEventFilters(expr);
+      fail("Unexpected not char. Exception should have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "(abc,def,uvc)) OR (rst, uvx)";
+    try {
+      TimelineReaderWebServicesUtils.parseEventFilters(expr);
+      fail("Unbalanced brackets. Exception should have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "(((! ,(abc,def,uvc) OR (rst, uvx)) AND (!(abcdefg) OR !(ghj,tyu" +
+        "))) OR ((bcd,tyu) AND uvb))";
+    try {
+      TimelineReaderWebServicesUtils.parseEventFilters(expr);
+      fail("( should follow ! char. Exception should have been thrown");
+    } catch (TimelineParseException e){}
+
+    assertNull(TimelineReaderWebServicesUtils.parseEventFilters(null));
+    assertNull(TimelineReaderWebServicesUtils.parseEventFilters("   "));
+  }
+
+  @Test
+  public void testRelationFiltersParsing() throws Exception {
+    String expr = "type1:entity11,type2:entity21:entity22";
+    TimelineFilterList expectedList = new TimelineFilterList(
+        new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL,
+            "type1", Sets.newHashSet((Object)"entity11")),
+        new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL,
+            "type2", Sets.newHashSet((Object)"entity21", "entity22"))
+    );
+    verifyFilterList(expr, TimelineReaderWebServicesUtils.
+        parseRelationFilters(expr), expectedList);
+
+    expr = "(type1:entity11,type2:entity21:entity22)";
+    verifyFilterList(expr, TimelineReaderWebServicesUtils.
+        parseRelationFilters(expr), expectedList);
+
+    expr = "(type1:entity11,type2:entity21:entity22) OR (type3:entity31:" +
+        "entity32:entity33,type1:entity11:entity12)";
+    expectedList = new TimelineFilterList(Operator.OR,
+        new TimelineFilterList(
+            new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL,
+                "type1", Sets.newHashSet((Object)"entity11")),
+            new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL,
+                "type2", Sets.newHashSet((Object)"entity21", "entity22"))
+        ),
+        new TimelineFilterList(
+            new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL,
+                "type3", Sets.newHashSet(
+                    (Object)"entity31", "entity32", "entity33")),
+            new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL,
+                "type1", Sets.newHashSet((Object)"entity11", "entity12"))
+        )
+    );
+    verifyFilterList(expr, TimelineReaderWebServicesUtils.
+        parseRelationFilters(expr), expectedList);
+
+    expr = "!(type1:entity11,type2:entity21:entity22,type5:entity51) OR " +
+        "(type3:entity31:entity32:entity33,type1:entity11:entity12)";
+
+    expectedList = new TimelineFilterList(Operator.OR,
+        new TimelineFilterList(
+            new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL,
+                "type1", Sets.newHashSet((Object)"entity11")),
+            new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL,
+                "type2", Sets.newHashSet((Object)"entity21", "entity22")),
+            new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL,
+                "type5", Sets.newHashSet((Object)"entity51"))
+        ),
+        new TimelineFilterList(
+            new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL,
+                "type3", Sets.newHashSet(
+                    (Object)"entity31", "entity32", "entity33")),
+            new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL,
+                "type1", Sets.newHashSet((Object)"entity11", "entity12"))
+        )
+    );
+    verifyFilterList(expr, TimelineReaderWebServicesUtils.
+        parseRelationFilters(expr), expectedList);
+
+    expr = "(((!(type1:entity11,type2:entity21:entity22,type5:entity51) OR " +
+        "(type3:entity31:entity32:entity33,type1:entity11:entity12)) AND "+
+        "(!(type11:entity111) OR !(type4:entity43:entity44:entity47:entity49," +
+        "type7:entity71))) OR ((type2:entity2,type8:entity88) AND t9:e:e1))";
+    expectedList = new TimelineFilterList(Operator.OR,
+        new TimelineFilterList(
+            new TimelineFilterList(Operator.OR,
+                new TimelineFilterList(
+                    new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL,
+                        "type1", Sets.newHashSet((Object)"entity11")),
+                    new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL,
+                        "type2", Sets.newHashSet(
+                            (Object)"entity21", "entity22")),
+                    new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL,
+                        "type5", Sets.newHashSet((Object)"entity51"))
+                ),
+                new TimelineFilterList(
+                    new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL,
+                        "type3", Sets.newHashSet(
+                            (Object)"entity31", "entity32", "entity33")),
+                    new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL,
+                        "type1", Sets.newHashSet(
+                            (Object)"entity11", "entity12"))
+                )
+            ),
+            new TimelineFilterList(Operator.OR,
+                new TimelineFilterList(
+                    new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL,
+                        "type11", Sets.newHashSet((Object)"entity111"))
+                ),
+                new TimelineFilterList(
+                    new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL,
+                        "type4", Sets.newHashSet((Object)"entity43","entity44",
+                            "entity47","entity49")),
+                    new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL,
+                        "type7", Sets.newHashSet((Object)"entity71"))
+                )
+            )
+        ),
+        new TimelineFilterList(
+            new TimelineFilterList(
+                new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL,
+                    "type2", Sets.newHashSet((Object)"entity2")),
+                new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL,
+                    "type8", Sets.newHashSet((Object)"entity88"))
+            ),
+            new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL, "t9",
+                Sets.newHashSet((Object)"e", "e1"))
+        )
+    );
+    verifyFilterList(expr, TimelineReaderWebServicesUtils.
+        parseRelationFilters(expr), expectedList);
+
+    expr = "   (   (  (   !   (   type1:entity11  ,  type2:entity21:entity22" +
+        "  ,  type5:entity51  )   OR  (   type3:entity31:entity32:entity33  " +
+        "     ,   type1:entity11:entity12)) AND (!(  type11:entity111  )  OR " +
+        "    !   (   type4:entity43:entity44:entity47:entity49 , " +
+        "type7:entity71  )  )  ) OR  (  (  type2:entity2 , type8:entity88) " +
+        "AND  t9:e:e1 )    ) ";
+    verifyFilterList(expr, TimelineReaderWebServicesUtils.
+        parseRelationFilters(expr), expectedList);
+
+    expr = "(((!(type1 : entity11,type2:entity21:entity22,type5:entity51) OR " +
+        "(type3:entity31:entity32:entity33,type1:entity11:entity12)) AND "+
+        "(!(type11:entity111) OR !(type4:entity43:entity44:entity47:entity49," +
+        "type7:entity71))) OR ((type2:entity2,type8:entity88) AND t9:e:e1))";
+    try {
+      TimelineReaderWebServicesUtils.parseRelationFilters(expr);
+      fail("Space not allowed in relation expression. Exception should have " +
+          "been thrown");
+    } catch (TimelineParseException e){}
+  }
+
+  @Test
+  public void testDataToRetrieve() throws Exception {
+    String expr = "abc,def";
+    TimelineFilterList expectedList = new TimelineFilterList(Operator.OR,
+        new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "abc"),
+        new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "def")
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList);
+
+    expr = "(abc,def)";
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList);
+
+    expr = "   (   abc  ,   def  )   ";
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList);
+
+    expr = "    abc  ,   def   ";
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList);
+
+    expr = "!(abc,def)";
+    expectedList = new TimelineFilterList(
+        new TimelinePrefixFilter(TimelineCompareOp.NOT_EQUAL, "abc"),
+        new TimelinePrefixFilter(TimelineCompareOp.NOT_EQUAL, "def")
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList);
+
+    expr = " !  (   abc  ,  def  )  ";
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList);
+
+    expr = "!(abc)";
+    expectedList = new TimelineFilterList(
+        new TimelinePrefixFilter(TimelineCompareOp.NOT_EQUAL, "abc")
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList);
+
+    expr = "(abc)";
+    expectedList = new TimelineFilterList(Operator.OR,
+        new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "abc")
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList);
+
+    expr = "abc";
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList);
+
+    expr = " !  (   abc  ,  def  , xyz)  ";
+    expectedList = new TimelineFilterList(
+        new TimelinePrefixFilter(TimelineCompareOp.NOT_EQUAL, "abc"),
+        new TimelinePrefixFilter(TimelineCompareOp.NOT_EQUAL, "def"),
+        new TimelinePrefixFilter(TimelineCompareOp.NOT_EQUAL, "xyz")
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList);
+
+    expr = "!(abc,def,xyz)";
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList);
+
+    expr = "!(abc,def,xyz";
+    try {
+      TimelineReaderWebServicesUtils.parseDataToRetrieve(expr);
+      fail("No closing bracket. Exception should have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "!abc,def,xyz";
+    try {
+      TimelineReaderWebServicesUtils.parseDataToRetrieve(expr);
+      fail("NOT(!) should be followed by opening bracket. Exception should " +
+          "have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "!abc,def,xyz";
+    try {
+      TimelineReaderWebServicesUtils.parseDataToRetrieve(expr);
+      fail("NOT(!) should be followed by opening bracket. Exception should " +
+          "have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "!   r(  abc,def,xyz)";
+    try {
+      TimelineReaderWebServicesUtils.parseDataToRetrieve(expr);
+      fail("NOT(!) should be followed by opening bracket. Exception should " +
+          "have been thrown");
+    } catch (TimelineParseException e){}
+
+    assertNull(TimelineReaderWebServicesUtils.parseDataToRetrieve(null));
+    assertNull(TimelineReaderWebServicesUtils.parseDataToRetrieve("     "));
+    assertNull(TimelineReaderWebServicesUtils.parseDataToRetrieve("()"));
+    assertNull(TimelineReaderWebServicesUtils.parseDataToRetrieve("!()"));
+    assertNull(TimelineReaderWebServicesUtils.parseDataToRetrieve("(     )"));
+    assertNull(TimelineReaderWebServicesUtils.parseDataToRetrieve("!(   )"));
+    assertNull(TimelineReaderWebServicesUtils.parseDataToRetrieve("!  (   )"));
+  }
+}

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