Преглед изворни кода

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

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

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

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

+ 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
 public class TimelineExistsFilter extends TimelineFilter {
 
-  private final TimelineCompareOp compareOp;
-  private final String value;
+  private TimelineCompareOp compareOp;
+  private String value;
+
+  public TimelineExistsFilter() {
+  }
 
   public TimelineExistsFilter(TimelineCompareOp op, String value) {
     this.value = value;
@@ -41,15 +44,57 @@ public class TimelineExistsFilter extends TimelineFilter {
     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
   public TimelineFilterType getFilterType() {
     return TimelineFilterType.EXISTS;
   }
 
+  public void setValue(String val) {
+    value = val;
+  }
+
   public String getValue() {
     return value;
   }
 
+  public void setCompareOp(TimelineCompareOp op) {
+    compareOp = op;
+  }
+
   public TimelineCompareOp getCompareOp() {
     return compareOp;
   }

+ 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);
   }
 
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result =
+        prime * result + ((filterList == null) ? 0 : filterList.hashCode());
+    result =
+        prime * result + ((operator == null) ? 0 : operator.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    TimelineFilterList other = (TimelineFilterList) obj;
+    if (operator != other.operator) {
+      return false;
+    }
+    if (filterList == null) {
+      if (other.filterList != null) {
+        return false;
+      }
+    } else if (!filterList.equals(other.filterList)) {
+      return false;
+    }
+    return true;
+  }
+
   @Override
   public String toString() {
     return String.format("TimelineFilterList %s (%d): %s",

+ 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
 @Unstable
 public class TimelineKeyValueFilter extends TimelineCompareFilter {
+  public TimelineKeyValueFilter() {
+  }
+
   public TimelineKeyValueFilter(TimelineCompareOp op, String key, Object val,
       boolean keyMustExistFlag) {
     super(op, key, val, keyMustExistFlag);
@@ -45,4 +48,14 @@ public class TimelineKeyValueFilter extends TimelineCompareFilter {
   public TimelineFilterType getFilterType() {
     return TimelineFilterType.KEY_VALUE;
   }
+
+  public void setCompareOp(TimelineCompareOp timelineCompareOp,
+      boolean keyExistFlag) {
+    if (timelineCompareOp != TimelineCompareOp.EQUAL &&
+        timelineCompareOp != TimelineCompareOp.NOT_EQUAL) {
+      throw new IllegalArgumentException("TimelineCompareOp for equality"
+          + " filter should be EQUAL or NOT_EQUAL");
+    }
+    super.setCompareOp(timelineCompareOp, keyExistFlag);
+  }
 }

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

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

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

Разлика између датотеке није приказан због своје велике величине
+ 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("!  (   )"));
+  }
+}

Неке датотеке нису приказане због велике количине промена