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

This commit is contained in:
Sangjin Lee 2016-05-02 14:06:19 -07:00
parent f0dbd7a40f
commit 089caf49fe
21 changed files with 3444 additions and 199 deletions

View File

@ -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 = ',';
}

View File

@ -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);
}
}

View File

@ -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;
}

View File

@ -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;
}
}

View File

@ -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() {
}
}

View File

@ -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;
}
}

View File

@ -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);
}
}

View File

@ -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);
}
}
}

View File

@ -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);
}
}
}

View File

@ -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);
}
}
}

View File

@ -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);
}
}

View File

@ -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));
}
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
* 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.
* 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 parseKeyStrValueStr(String str, String pairsDelim,
String keyValDelim) {
return parseKeyValues(str, pairsDelim, keyValDelim, true, false);
}
/**
* 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.
*/
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));
}
}

View File

@ -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)",

View File

@ -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;
}

View File

@ -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",

View File

@ -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);
}
}

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -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>>(){});

View File

@ -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("! ( )"));
}
}