diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g b/nifi/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g index 10394b9902..80581f5c01 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g +++ b/nifi/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g @@ -150,6 +150,7 @@ TO_RADIX : 'toRadix'; OR : 'or'; AND : 'and'; JOIN : 'join'; +TO_LITERAL : 'literal'; // 2 arg functions SUBSTRING : 'substring'; diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g b/nifi/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g index f6a87ddcc8..7c37530a4d 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g +++ b/nifi/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g @@ -127,7 +127,8 @@ functionCall : functionRef -> booleanLiteral : TRUE | FALSE; zeroArgStandaloneFunction : (IP | UUID | NOW | NEXT_INT | HOSTNAME) LPAREN! RPAREN!; -oneArgStandaloneFunction : HOSTNAME^ LPAREN! booleanLiteral RPAREN!; +oneArgStandaloneFunction : (TO_LITERAL^ LPAREN! anyArg RPAREN!) | + (HOSTNAME^ LPAREN! booleanLiteral RPAREN!); standaloneFunction : zeroArgStandaloneFunction | oneArgStandaloneFunction; attributeRefOrFunctionCall : (attributeRef | standaloneFunction); diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java index 77d08c6c4a..07fbac674d 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java @@ -95,12 +95,14 @@ import org.apache.nifi.attribute.expression.language.evaluation.functions.UuidEv import org.apache.nifi.attribute.expression.language.evaluation.literals.BooleanLiteralEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.literals.NumberLiteralEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.literals.StringLiteralEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.literals.ToLiteralEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.reduce.CountEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.reduce.JoinEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.reduce.ReduceEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.selection.AllAttributesEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.selection.AnyAttributeEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.selection.DelineatedAttributeEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.selection.IteratingEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiAttributeEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiMatchAttributeEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiNamedAttributeEvaluator; @@ -114,6 +116,7 @@ import org.antlr.runtime.ANTLRStringStream; import org.antlr.runtime.CharStream; import org.antlr.runtime.CommonTokenStream; import org.antlr.runtime.tree.Tree; + import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ALL_ATTRIBUTES; import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ALL_DELINEATED_VALUES; import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ALL_MATCHING_ATTRIBUTES; @@ -171,6 +174,7 @@ import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpre import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_BEFORE; import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_BEFORE_LAST; import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_DATE; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_LITERAL; import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_LOWER; import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_NUMBER; import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_RADIX; @@ -181,6 +185,7 @@ import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpre import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.URL_DECODE; import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.URL_ENCODE; import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UUID; + import org.apache.nifi.attribute.expression.language.evaluation.selection.MappingEvaluator; /** @@ -205,7 +210,7 @@ public class Query { try { validateExpression(value, false); return true; - } catch (final ProcessException e) { + } catch (final AttributeExpressionLanguageParsingException | ProcessException e) { return false; } } @@ -341,7 +346,7 @@ public class Query { if (c == '\\') { backslashCount++; - } else if (c == quoteChar && ((backslashCount % 2 == 0) || lastChar != '\\')) { + } else if (c == quoteChar && (backslashCount % 2 == 0 || lastChar != '\\')) { return i; } @@ -359,7 +364,7 @@ public class Query { final String value = evaluated.toString(); final String escaped = value.replace("$$", "$"); - return (decorator == null) ? escaped : decorator.decorate(escaped); + return decorator == null ? escaped : decorator.decorate(escaped); } static String evaluateExpressions(final String rawValue, Map expressionMap) throws ProcessException { @@ -631,6 +636,25 @@ public class Query { } private static void verifyMappingEvaluatorReduced(final Evaluator evaluator) { + final Evaluator rightMostEvaluator; + if (evaluator instanceof IteratingEvaluator) { + rightMostEvaluator = ((IteratingEvaluator) evaluator).getLogicEvaluator(); + } else { + rightMostEvaluator = evaluator; + } + + Evaluator eval = rightMostEvaluator.getSubjectEvaluator(); + Evaluator lastEval = rightMostEvaluator; + while (eval != null) { + if (eval instanceof ReduceEvaluator) { + throw new AttributeExpressionLanguageParsingException("Expression attempts to call function '" + lastEval.getToken() + "' on the result of '" + eval.getToken() + + "'. This is not allowed. Instead, use \"${literal( ${} ):" + lastEval.getToken() + "(...)}\""); + } + + lastEval = eval; + eval = eval.getSubjectEvaluator(); + } + // if the result type of the evaluator is BOOLEAN, then it will always // be reduced when evaluator. final ResultType resultType = evaluator.getResultType(); @@ -691,7 +715,7 @@ public class Query { return "Query [" + query + "]"; } - private static StringEvaluator newStringLiteralEvaluator(final String literalValue) { + private static Evaluator newStringLiteralEvaluator(final String literalValue) { if (literalValue == null || literalValue.length() < 2) { return new StringLiteralEvaluator(literalValue); } @@ -724,7 +748,7 @@ public class Query { return toStringEvaluator(evaluators.get(0)); } - StringEvaluator lastEvaluator = toStringEvaluator(evaluators.get(0)); + Evaluator lastEvaluator = toStringEvaluator(evaluators.get(0)); for (int i = 1; i < evaluators.size(); i++) { lastEvaluator = new AppendEvaluator(lastEvaluator, toStringEvaluator(evaluators.get(i))); } @@ -749,8 +773,8 @@ public class Query { final Tree functionTypeTree = tree.getChild(0); final int multiAttrType = functionTypeTree.getType(); if (multiAttrType == ANY_DELINEATED_VALUE || multiAttrType == ALL_DELINEATED_VALUES) { - final StringEvaluator delineatedValueEvaluator = toStringEvaluator(buildEvaluator(tree.getChild(1))); - final StringEvaluator delimiterEvaluator = toStringEvaluator(buildEvaluator(tree.getChild(2))); + final Evaluator delineatedValueEvaluator = toStringEvaluator(buildEvaluator(tree.getChild(1))); + final Evaluator delimiterEvaluator = toStringEvaluator(buildEvaluator(tree.getChild(2))); return new DelineatedAttributeEvaluator(delineatedValueEvaluator, delimiterEvaluator, multiAttrType); } @@ -807,6 +831,10 @@ public class Query { case NOW: { return new NowEvaluator(); } + case TO_LITERAL: { + final Evaluator argEvaluator = buildEvaluator(tree.getChild(0)); + return new ToLiteralEvaluator(argEvaluator); + } case IP: { try { return new IPEvaluator(); @@ -818,7 +846,7 @@ public class Query { if (tree.getChildCount() == 0) { try { return new HostnameEvaluator(false); - } catch (UnknownHostException e) { + } catch (final UnknownHostException e) { throw new AttributeExpressionLanguageException(e); } } else if (tree.getChildCount() == 1) { @@ -832,7 +860,7 @@ public class Query { default: throw new AttributeExpressionLanguageParsingException("Call to hostname() must take 0 or 1 (boolean) parameter"); } - } catch (UnknownHostException e) { + } catch (final UnknownHostException e) { throw new AttributeExpressionLanguageException(e); } } else { @@ -847,16 +875,22 @@ public class Query { } } + private static Evaluator addToken(final Evaluator evaluator, final String token) { + evaluator.setToken(token); + return evaluator; + } + private static Evaluator buildBooleanEvaluator(final Tree tree) { switch (tree.getType()) { case TRUE: - return new BooleanLiteralEvaluator(true); + return addToken(new BooleanLiteralEvaluator(true), "true"); case FALSE: - return new BooleanLiteralEvaluator(false); + return addToken(new BooleanLiteralEvaluator(false), "true"); } throw new AttributeExpressionLanguageParsingException("Cannot build Boolean evaluator from tree " + tree.toString()); } + @SuppressWarnings({ "rawtypes", "unchecked" }) private static Evaluator buildExpressionEvaluator(final Tree tree) { if (tree.getChildCount() == 0) { throw new AttributeExpressionLanguageParsingException("EXPRESSION tree node has no children"); @@ -901,6 +935,27 @@ public class Query { break; } } + + switch (multiAttrEval.getEvaluationType()) { + case ANY_ATTRIBUTE: + chosenEvaluator.setToken("anyAttribute"); + break; + case ANY_MATCHING_ATTRIBUTE: + chosenEvaluator.setToken("anyMatchingAttribute"); + break; + case ANY_DELINEATED_VALUE: + chosenEvaluator.setToken("anyDelineatedValue"); + break; + case ALL_ATTRIBUTES: + chosenEvaluator.setToken("allAttributes"); + break; + case ALL_MATCHING_ATTRIBUTES: + chosenEvaluator.setToken("allMatchingAttributes"); + break; + case ALL_DELINEATED_VALUES: + chosenEvaluator.setToken("allDelineatedValues"); + break; + } } } @@ -934,24 +989,25 @@ public class Query { return args; } - private static StringEvaluator toStringEvaluator(final Evaluator evaluator) { + private static Evaluator toStringEvaluator(final Evaluator evaluator) { return toStringEvaluator(evaluator, null); } - private static StringEvaluator toStringEvaluator(final Evaluator evaluator, final String location) { + private static Evaluator toStringEvaluator(final Evaluator evaluator, final String location) { if (evaluator.getResultType() == ResultType.STRING) { return (StringEvaluator) evaluator; } - return new StringCastEvaluator(evaluator); + return addToken(new StringCastEvaluator(evaluator), evaluator.getToken()); } - private static BooleanEvaluator toBooleanEvaluator(final Evaluator evaluator, final String location) { + @SuppressWarnings("unchecked") + private static Evaluator toBooleanEvaluator(final Evaluator evaluator, final String location) { switch (evaluator.getResultType()) { case BOOLEAN: - return (BooleanEvaluator) evaluator; + return (Evaluator) evaluator; case STRING: - return new BooleanCastEvaluator((StringEvaluator) evaluator); + return addToken(new BooleanCastEvaluator((StringEvaluator) evaluator), evaluator.getToken()); default: throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + evaluator.getResultType() + " to " + ResultType.BOOLEAN + (location == null ? "" : " at location [" + location + "]")); @@ -959,22 +1015,23 @@ public class Query { } - private static BooleanEvaluator toBooleanEvaluator(final Evaluator evaluator) { + private static Evaluator toBooleanEvaluator(final Evaluator evaluator) { return toBooleanEvaluator(evaluator, null); } - private static NumberEvaluator toNumberEvaluator(final Evaluator evaluator) { + private static Evaluator toNumberEvaluator(final Evaluator evaluator) { return toNumberEvaluator(evaluator, null); } - private static NumberEvaluator toNumberEvaluator(final Evaluator evaluator, final String location) { + @SuppressWarnings("unchecked") + private static Evaluator toNumberEvaluator(final Evaluator evaluator, final String location) { switch (evaluator.getResultType()) { case NUMBER: - return (NumberEvaluator) evaluator; + return (Evaluator) evaluator; case STRING: - return new NumberCastEvaluator(evaluator); + return addToken(new NumberCastEvaluator(evaluator), evaluator.getToken()); case DATE: - return new DateToNumberEvaluator((DateEvaluator) evaluator); + return addToken(new DateToNumberEvaluator((DateEvaluator) evaluator), evaluator.getToken()); default: throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + evaluator.getResultType() + " to " + ResultType.NUMBER + (location == null ? "" : " at location [" + location + "]")); @@ -997,233 +1054,233 @@ public class Query { switch (tree.getType()) { case TRIM: { verifyArgCount(argEvaluators, 0, "trim"); - return new TrimEvaluator(toStringEvaluator(subjectEvaluator)); + return addToken(new TrimEvaluator(toStringEvaluator(subjectEvaluator)), "trim"); } case TO_STRING: { verifyArgCount(argEvaluators, 0, "toString"); - return new ToStringEvaluator(subjectEvaluator); + return addToken(new ToStringEvaluator(subjectEvaluator), "toString"); } case TO_LOWER: { verifyArgCount(argEvaluators, 0, "toLower"); - return new ToLowerEvaluator(toStringEvaluator(subjectEvaluator)); + return addToken(new ToLowerEvaluator(toStringEvaluator(subjectEvaluator)), "toLower"); } case TO_UPPER: { verifyArgCount(argEvaluators, 0, "toUpper"); - return new ToUpperEvaluator(toStringEvaluator(subjectEvaluator)); + return addToken(new ToUpperEvaluator(toStringEvaluator(subjectEvaluator)), "toUpper"); } case URL_ENCODE: { verifyArgCount(argEvaluators, 0, "urlEncode"); - return new UrlEncodeEvaluator(toStringEvaluator(subjectEvaluator)); + return addToken(new UrlEncodeEvaluator(toStringEvaluator(subjectEvaluator)), "urlEncode"); } case URL_DECODE: { verifyArgCount(argEvaluators, 0, "urlDecode"); - return new UrlDecodeEvaluator(toStringEvaluator(subjectEvaluator)); + return addToken(new UrlDecodeEvaluator(toStringEvaluator(subjectEvaluator)), "urlDecode"); } case SUBSTRING_BEFORE: { verifyArgCount(argEvaluators, 1, "substringBefore"); - return new SubstringBeforeEvaluator(toStringEvaluator(subjectEvaluator), - toStringEvaluator(argEvaluators.get(0), "first argument to substringBefore")); + return addToken(new SubstringBeforeEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to substringBefore")), "substringBefore"); } case SUBSTRING_BEFORE_LAST: { verifyArgCount(argEvaluators, 1, "substringBeforeLast"); - return new SubstringBeforeLastEvaluator(toStringEvaluator(subjectEvaluator), - toStringEvaluator(argEvaluators.get(0), "first argument to substringBeforeLast")); + return addToken(new SubstringBeforeLastEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to substringBeforeLast")), "substringBeforeLast"); } case SUBSTRING_AFTER: { verifyArgCount(argEvaluators, 1, "substringAfter"); - return new SubstringAfterEvaluator(toStringEvaluator(subjectEvaluator), - toStringEvaluator(argEvaluators.get(0), "first argument to substringAfter")); + return addToken(new SubstringAfterEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to substringAfter")), "substringAfter"); } case SUBSTRING_AFTER_LAST: { verifyArgCount(argEvaluators, 1, "substringAfterLast"); - return new SubstringAfterLastEvaluator(toStringEvaluator(subjectEvaluator), - toStringEvaluator(argEvaluators.get(0), "first argument to substringAfterLast")); + return addToken(new SubstringAfterLastEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to substringAfterLast")), "substringAfterLast"); } case REPLACE_NULL: { verifyArgCount(argEvaluators, 1, "replaceNull"); - return new ReplaceNullEvaluator(toStringEvaluator(subjectEvaluator), - toStringEvaluator(argEvaluators.get(0), "first argument to replaceNull")); + return addToken(new ReplaceNullEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to replaceNull")), "replaceNull"); } case REPLACE_EMPTY: { verifyArgCount(argEvaluators, 1, "replaceEmtpy"); - return new ReplaceEmptyEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0), "first argumen to replaceEmpty")); + return addToken(new ReplaceEmptyEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0), "first argument to replaceEmpty")), "replaceEmpty"); } case REPLACE: { verifyArgCount(argEvaluators, 2, "replace"); - return new ReplaceEvaluator(toStringEvaluator(subjectEvaluator), + return addToken(new ReplaceEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0), "first argument to replace"), - toStringEvaluator(argEvaluators.get(1), "second argument to replace")); + toStringEvaluator(argEvaluators.get(1), "second argument to replace")), "replace"); } case REPLACE_ALL: { verifyArgCount(argEvaluators, 2, "replaceAll"); - return new ReplaceAllEvaluator(toStringEvaluator(subjectEvaluator), + return addToken(new ReplaceAllEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0), "first argument to replaceAll"), - toStringEvaluator(argEvaluators.get(1), "second argument to replaceAll")); + toStringEvaluator(argEvaluators.get(1), "second argument to replaceAll")), "replaceAll"); } case APPEND: { verifyArgCount(argEvaluators, 1, "append"); - return new AppendEvaluator(toStringEvaluator(subjectEvaluator), - toStringEvaluator(argEvaluators.get(0), "first argument to append")); + return addToken(new AppendEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to append")), "append"); } case PREPEND: { verifyArgCount(argEvaluators, 1, "prepend"); - return new PrependEvaluator(toStringEvaluator(subjectEvaluator), - toStringEvaluator(argEvaluators.get(0), "first argument to prepend")); + return addToken(new PrependEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to prepend")), "prepend"); } case SUBSTRING: { final int numArgs = argEvaluators.size(); if (numArgs == 1) { - return new SubstringEvaluator(toStringEvaluator(subjectEvaluator), - toNumberEvaluator(argEvaluators.get(0), "first argument to substring")); + return addToken(new SubstringEvaluator(toStringEvaluator(subjectEvaluator), + toNumberEvaluator(argEvaluators.get(0), "first argument to substring")), "substring"); } else if (numArgs == 2) { - return new SubstringEvaluator(toStringEvaluator(subjectEvaluator), + return addToken(new SubstringEvaluator(toStringEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0), "first argument to substring"), - toNumberEvaluator(argEvaluators.get(1), "second argument to substring")); + toNumberEvaluator(argEvaluators.get(1), "second argument to substring")), "substring"); } else { throw new AttributeExpressionLanguageParsingException("substring() function can take either 1 or 2 arguments but cannot take " + numArgs + " arguments"); } } case JOIN: { verifyArgCount(argEvaluators, 1, "join"); - return new JoinEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0))); + return addToken(new JoinEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0))), "join"); } case COUNT: { verifyArgCount(argEvaluators, 0, "count"); - return new CountEvaluator(subjectEvaluator); + return addToken(new CountEvaluator(subjectEvaluator), "count"); } case IS_NULL: { verifyArgCount(argEvaluators, 0, "isNull"); - return new IsNullEvaluator(toStringEvaluator(subjectEvaluator)); + return addToken(new IsNullEvaluator(toStringEvaluator(subjectEvaluator)), "isNull"); } case IS_EMPTY: { verifyArgCount(argEvaluators, 0, "isNull"); - return new IsEmptyEvaluator(toStringEvaluator(subjectEvaluator)); + return addToken(new IsEmptyEvaluator(toStringEvaluator(subjectEvaluator)), "isEmpty"); } case NOT_NULL: { verifyArgCount(argEvaluators, 0, "notNull"); - return new NotNullEvaluator(toStringEvaluator(subjectEvaluator)); + return addToken(new NotNullEvaluator(toStringEvaluator(subjectEvaluator)), "notNull"); } case STARTS_WITH: { verifyArgCount(argEvaluators, 1, "startsWith"); - return new StartsWithEvaluator(toStringEvaluator(subjectEvaluator), - toStringEvaluator(argEvaluators.get(0), "first argument to startsWith")); + return addToken(new StartsWithEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to startsWith")), "startsWith"); } case ENDS_WITH: { verifyArgCount(argEvaluators, 1, "endsWith"); - return new EndsWithEvaluator(toStringEvaluator(subjectEvaluator), - toStringEvaluator(argEvaluators.get(0), "first argument to endsWith")); + return addToken(new EndsWithEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to endsWith")), "endsWith"); } case CONTAINS: { verifyArgCount(argEvaluators, 1, "contains"); - return new ContainsEvaluator(toStringEvaluator(subjectEvaluator), - toStringEvaluator(argEvaluators.get(0), "first argument to contains")); + return addToken(new ContainsEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to contains")), "contains"); } case FIND: { verifyArgCount(argEvaluators, 1, "find"); - return new FindEvaluator(toStringEvaluator(subjectEvaluator), - toStringEvaluator(argEvaluators.get(0), "first argument to find")); + return addToken(new FindEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to find")), "find"); } case MATCHES: { verifyArgCount(argEvaluators, 1, "matches"); - return new MatchesEvaluator(toStringEvaluator(subjectEvaluator), - toStringEvaluator(argEvaluators.get(0), "first argument to matches")); + return addToken(new MatchesEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to matches")), "matches"); } case EQUALS: { verifyArgCount(argEvaluators, 1, "equals"); - return new EqualsEvaluator(subjectEvaluator, argEvaluators.get(0)); + return addToken(new EqualsEvaluator(subjectEvaluator, argEvaluators.get(0)), "equals"); } case EQUALS_IGNORE_CASE: { verifyArgCount(argEvaluators, 1, "equalsIgnoreCase"); - return new EqualsIgnoreCaseEvaluator(toStringEvaluator(subjectEvaluator), - toStringEvaluator(argEvaluators.get(0), "first argument to equalsIgnoreCase")); + return addToken(new EqualsIgnoreCaseEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to equalsIgnoreCase")), "equalsIgnoreCase"); } case GREATER_THAN: { verifyArgCount(argEvaluators, 1, "gt"); - return new GreaterThanEvaluator(toNumberEvaluator(subjectEvaluator), - toNumberEvaluator(argEvaluators.get(0), "first argument to gt")); + return addToken(new GreaterThanEvaluator(toNumberEvaluator(subjectEvaluator), + toNumberEvaluator(argEvaluators.get(0), "first argument to gt")), "gt"); } case GREATER_THAN_OR_EQUAL: { verifyArgCount(argEvaluators, 1, "ge"); - return new GreaterThanOrEqualEvaluator(toNumberEvaluator(subjectEvaluator), - toNumberEvaluator(argEvaluators.get(0), "first argument to ge")); + return addToken(new GreaterThanOrEqualEvaluator(toNumberEvaluator(subjectEvaluator), + toNumberEvaluator(argEvaluators.get(0), "first argument to ge")), "ge"); } case LESS_THAN: { verifyArgCount(argEvaluators, 1, "lt"); - return new LessThanEvaluator(toNumberEvaluator(subjectEvaluator), - toNumberEvaluator(argEvaluators.get(0), "first argument to lt")); + return addToken(new LessThanEvaluator(toNumberEvaluator(subjectEvaluator), + toNumberEvaluator(argEvaluators.get(0), "first argument to lt")), "lt"); } case LESS_THAN_OR_EQUAL: { verifyArgCount(argEvaluators, 1, "le"); - return new LessThanOrEqualEvaluator(toNumberEvaluator(subjectEvaluator), - toNumberEvaluator(argEvaluators.get(0), "first argument to le")); + return addToken(new LessThanOrEqualEvaluator(toNumberEvaluator(subjectEvaluator), + toNumberEvaluator(argEvaluators.get(0), "first argument to le")), "le"); } case LENGTH: { verifyArgCount(argEvaluators, 0, "length"); - return new LengthEvaluator(toStringEvaluator(subjectEvaluator)); + return addToken(new LengthEvaluator(toStringEvaluator(subjectEvaluator)), "length"); } case TO_DATE: { if (argEvaluators.isEmpty()) { - return new NumberToDateEvaluator(toNumberEvaluator(subjectEvaluator)); + return addToken(new NumberToDateEvaluator(toNumberEvaluator(subjectEvaluator)), "toDate"); } else if (subjectEvaluator.getResultType() == ResultType.STRING) { - return new StringToDateEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0))); + return addToken(new StringToDateEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0))), "toDate"); } else { - return new NumberToDateEvaluator(toNumberEvaluator(subjectEvaluator)); + return addToken(new NumberToDateEvaluator(toNumberEvaluator(subjectEvaluator)), "toDate"); } } case TO_NUMBER: { verifyArgCount(argEvaluators, 0, "toNumber"); switch (subjectEvaluator.getResultType()) { case STRING: - return new ToNumberEvaluator((StringEvaluator) subjectEvaluator); + return addToken(new ToNumberEvaluator((StringEvaluator) subjectEvaluator), "toNumber"); case DATE: - return new DateToNumberEvaluator((DateEvaluator) subjectEvaluator); + return addToken(new DateToNumberEvaluator((DateEvaluator) subjectEvaluator), "toNumber"); default: throw new AttributeExpressionLanguageParsingException(subjectEvaluator + " returns type " + subjectEvaluator.getResultType() + " but expected to get " + ResultType.STRING); } } case TO_RADIX: { if (argEvaluators.size() == 1) { - return new ToRadixEvaluator((NumberEvaluator) subjectEvaluator, toNumberEvaluator(argEvaluators.get(0))); + return addToken(new ToRadixEvaluator((NumberEvaluator) subjectEvaluator, toNumberEvaluator(argEvaluators.get(0))), "toRadix"); } else { - return new ToRadixEvaluator((NumberEvaluator) subjectEvaluator, toNumberEvaluator(argEvaluators.get(0)), toNumberEvaluator(argEvaluators.get(1))); + return addToken(new ToRadixEvaluator((NumberEvaluator) subjectEvaluator, toNumberEvaluator(argEvaluators.get(0)), toNumberEvaluator(argEvaluators.get(1))), "toRadix"); } } case MOD: { - return new ModEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))); + return addToken(new ModEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "mod"); } case PLUS: { - return new PlusEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))); + return addToken(new PlusEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "plus"); } case MINUS: { - return new MinusEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))); + return addToken(new MinusEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "minus"); } case MULTIPLY: { - return new MultiplyEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))); + return addToken(new MultiplyEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "multiply"); } case DIVIDE: { - return new DivideEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))); + return addToken(new DivideEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "divide"); } case INDEX_OF: { verifyArgCount(argEvaluators, 1, "indexOf"); - return new IndexOfEvaluator(toStringEvaluator(subjectEvaluator), - toStringEvaluator(argEvaluators.get(0), "first argument to indexOf")); + return addToken(new IndexOfEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to indexOf")), "indexOf"); } case LAST_INDEX_OF: { verifyArgCount(argEvaluators, 1, "lastIndexOf"); - return new LastIndexOfEvaluator(toStringEvaluator(subjectEvaluator), - toStringEvaluator(argEvaluators.get(0), "first argument to lastIndexOf")); + return addToken(new LastIndexOfEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to lastIndexOf")), "lastIndexOf"); } case FORMAT: { - return new FormatEvaluator(toDateEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0), "first argument of format")); + return addToken(new FormatEvaluator(toDateEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0), "first argument of format")), "format"); } case OR: { - return new OrEvaluator(toBooleanEvaluator(subjectEvaluator), toBooleanEvaluator(argEvaluators.get(0))); + return addToken(new OrEvaluator(toBooleanEvaluator(subjectEvaluator), toBooleanEvaluator(argEvaluators.get(0))), "or"); } case AND: { - return new AndEvaluator(toBooleanEvaluator(subjectEvaluator), toBooleanEvaluator(argEvaluators.get(0))); + return addToken(new AndEvaluator(toBooleanEvaluator(subjectEvaluator), toBooleanEvaluator(argEvaluators.get(0))), "and"); } case NOT: { - return new NotEvaluator(toBooleanEvaluator(subjectEvaluator)); + return addToken(new NotEvaluator(toBooleanEvaluator(subjectEvaluator)), "not"); } default: throw new AttributeExpressionLanguageParsingException("Expected a Function-type expression but got " + tree.toString()); diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/BooleanEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/BooleanEvaluator.java index 376ddfe3d7..907ee9592b 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/BooleanEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/BooleanEvaluator.java @@ -19,6 +19,7 @@ package org.apache.nifi.attribute.expression.language.evaluation; import org.apache.nifi.expression.AttributeExpression.ResultType; public abstract class BooleanEvaluator implements Evaluator { + private String token; @Override public ResultType getResultType() { @@ -29,4 +30,14 @@ public abstract class BooleanEvaluator implements Evaluator { public int getEvaluationsRemaining() { return 0; } + + @Override + public String getToken() { + return token; + } + + @Override + public void setToken(final String token) { + this.token = token; + } } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateEvaluator.java index 7474b60077..caf3117568 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateEvaluator.java @@ -21,6 +21,7 @@ import java.util.Date; import org.apache.nifi.expression.AttributeExpression.ResultType; public abstract class DateEvaluator implements Evaluator { + private String token; @Override public ResultType getResultType() { @@ -31,4 +32,14 @@ public abstract class DateEvaluator implements Evaluator { public int getEvaluationsRemaining() { return 0; } + + @Override + public String getToken() { + return token; + } + + @Override + public void setToken(final String token) { + this.token = token; + } } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/Evaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/Evaluator.java index 6d164df25e..86f95a80de 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/Evaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/Evaluator.java @@ -29,4 +29,16 @@ public interface Evaluator { int getEvaluationsRemaining(); Evaluator getSubjectEvaluator(); + + /** + * Sets the token that was used in the query to cause this Evaluator to be created + * + * @param token the token that caused this Evaluator to be created + */ + void setToken(String token); + + /** + * @return the token that caused this Evaluator to be created + */ + String getToken(); } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberEvaluator.java index 403bae358e..ec3adace64 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberEvaluator.java @@ -19,6 +19,7 @@ package org.apache.nifi.attribute.expression.language.evaluation; import org.apache.nifi.expression.AttributeExpression.ResultType; public abstract class NumberEvaluator implements Evaluator { + private String token; @Override public ResultType getResultType() { @@ -30,4 +31,13 @@ public abstract class NumberEvaluator implements Evaluator { return 0; } + @Override + public String getToken() { + return token; + } + + @Override + public void setToken(final String token) { + this.token = token; + } } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/StringEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/StringEvaluator.java index 1f4ff21393..c6f2b7963c 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/StringEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/StringEvaluator.java @@ -19,6 +19,7 @@ package org.apache.nifi.attribute.expression.language.evaluation; import org.apache.nifi.expression.AttributeExpression.ResultType; public abstract class StringEvaluator implements Evaluator { + private String token; @Override public ResultType getResultType() { @@ -29,4 +30,14 @@ public abstract class StringEvaluator implements Evaluator { public int getEvaluationsRemaining() { return 0; } + + @Override + public String getToken() { + return token; + } + + @Override + public void setToken(final String token) { + this.token = token; + } } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AndEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AndEvaluator.java index 9ed8f06516..232fc2677f 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AndEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AndEvaluator.java @@ -25,10 +25,10 @@ import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; public class AndEvaluator extends BooleanEvaluator { - private final BooleanEvaluator subjectEvaluator; - private final BooleanEvaluator rhsEvaluator; + private final Evaluator subjectEvaluator; + private final Evaluator rhsEvaluator; - public AndEvaluator(final BooleanEvaluator subjectEvaluator, final BooleanEvaluator rhsEvaluator) { + public AndEvaluator(final Evaluator subjectEvaluator, final Evaluator rhsEvaluator) { this.subjectEvaluator = subjectEvaluator; this.rhsEvaluator = rhsEvaluator; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AppendEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AppendEvaluator.java index bd5e81a087..80f19744cb 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AppendEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AppendEvaluator.java @@ -25,10 +25,10 @@ import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResul public class AppendEvaluator extends StringEvaluator { - private final StringEvaluator subject; - private final StringEvaluator appendEvaluator; + private final Evaluator subject; + private final Evaluator appendEvaluator; - public AppendEvaluator(final StringEvaluator subject, final StringEvaluator append) { + public AppendEvaluator(final Evaluator subject, final Evaluator append) { this.subject = subject; this.appendEvaluator = append; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AttributeEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AttributeEvaluator.java index 1b4b74ecce..a0695a9457 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AttributeEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AttributeEvaluator.java @@ -25,9 +25,9 @@ import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResul public class AttributeEvaluator extends StringEvaluator { - private final StringEvaluator nameEvaluator; + private final Evaluator nameEvaluator; - public AttributeEvaluator(final StringEvaluator nameEvaluator) { + public AttributeEvaluator(final Evaluator nameEvaluator) { this.nameEvaluator = nameEvaluator; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ContainsEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ContainsEvaluator.java index 99802f6347..54170ba2e3 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ContainsEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ContainsEvaluator.java @@ -22,14 +22,13 @@ import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; -import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; public class ContainsEvaluator extends BooleanEvaluator { - private final StringEvaluator subject; - private final StringEvaluator search; + private final Evaluator subject; + private final Evaluator search; - public ContainsEvaluator(final StringEvaluator subject, final StringEvaluator search) { + public ContainsEvaluator(final Evaluator subject, final Evaluator search) { this.subject = subject; this.search = search; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/DivideEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/DivideEvaluator.java index c7583cbb75..12ab37f1d3 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/DivideEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/DivideEvaluator.java @@ -25,10 +25,10 @@ import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; public class DivideEvaluator extends NumberEvaluator { - private final NumberEvaluator subject; - private final NumberEvaluator divideValue; + private final Evaluator subject; + private final Evaluator divideValue; - public DivideEvaluator(final NumberEvaluator subject, final NumberEvaluator divideValue) { + public DivideEvaluator(final Evaluator subject, final Evaluator divideValue) { this.subject = subject; this.divideValue = divideValue; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EndsWithEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EndsWithEvaluator.java index c31c2b1230..13bc3966ac 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EndsWithEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EndsWithEvaluator.java @@ -22,14 +22,13 @@ import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; -import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; public class EndsWithEvaluator extends BooleanEvaluator { - private final StringEvaluator subject; - private final StringEvaluator search; + private final Evaluator subject; + private final Evaluator search; - public EndsWithEvaluator(final StringEvaluator subject, final StringEvaluator search) { + public EndsWithEvaluator(final Evaluator subject, final Evaluator search) { this.subject = subject; this.search = search; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FindEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FindEvaluator.java index b377c4ad3e..2a3f1aed59 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FindEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FindEvaluator.java @@ -23,17 +23,16 @@ import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; -import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.literals.StringLiteralEvaluator; public class FindEvaluator extends BooleanEvaluator { - private final StringEvaluator subject; - private final StringEvaluator search; + private final Evaluator subject; + private final Evaluator search; private final Pattern compiledPattern; - public FindEvaluator(final StringEvaluator subject, final StringEvaluator search) { + public FindEvaluator(final Evaluator subject, final Evaluator search) { this.subject = subject; this.search = search; diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FormatEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FormatEvaluator.java index 132b9e7d1c..717cbd5f3f 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FormatEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FormatEvaluator.java @@ -30,9 +30,9 @@ import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResul public class FormatEvaluator extends StringEvaluator { private final DateEvaluator subject; - private final StringEvaluator format; + private final Evaluator format; - public FormatEvaluator(final DateEvaluator subject, final StringEvaluator format) { + public FormatEvaluator(final DateEvaluator subject, final Evaluator format) { this.subject = subject; this.format = format; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanEvaluator.java index af1ee1d13c..8850225e4b 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanEvaluator.java @@ -21,15 +21,14 @@ import java.util.Map; import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; -import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; public class GreaterThanEvaluator extends BooleanEvaluator { - private final NumberEvaluator subject; - private final NumberEvaluator comparison; + private final Evaluator subject; + private final Evaluator comparison; - public GreaterThanEvaluator(final NumberEvaluator subject, final NumberEvaluator comparison) { + public GreaterThanEvaluator(final Evaluator subject, final Evaluator comparison) { this.subject = subject; this.comparison = comparison; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanOrEqualEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanOrEqualEvaluator.java index 1269fc0b0a..ff71335361 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanOrEqualEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanOrEqualEvaluator.java @@ -21,15 +21,14 @@ import java.util.Map; import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; -import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; public class GreaterThanOrEqualEvaluator extends BooleanEvaluator { - private final NumberEvaluator subject; - private final NumberEvaluator comparison; + private final Evaluator subject; + private final Evaluator comparison; - public GreaterThanOrEqualEvaluator(final NumberEvaluator subject, final NumberEvaluator comparison) { + public GreaterThanOrEqualEvaluator(final Evaluator subject, final Evaluator comparison) { this.subject = subject; this.comparison = comparison; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IndexOfEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IndexOfEvaluator.java index 80422cb67b..c4bc03eb6e 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IndexOfEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IndexOfEvaluator.java @@ -22,14 +22,13 @@ import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult; import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; -import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; public class IndexOfEvaluator extends NumberEvaluator { - private final StringEvaluator subject; - private final StringEvaluator indexEvaluator; + private final Evaluator subject; + private final Evaluator indexEvaluator; - public IndexOfEvaluator(final StringEvaluator subject, final StringEvaluator indexEvaluator) { + public IndexOfEvaluator(final Evaluator subject, final Evaluator indexEvaluator) { this.subject = subject; this.indexEvaluator = indexEvaluator; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LastIndexOfEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LastIndexOfEvaluator.java index 2b670935b1..3ebbee7685 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LastIndexOfEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LastIndexOfEvaluator.java @@ -22,14 +22,13 @@ import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult; import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; -import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; public class LastIndexOfEvaluator extends NumberEvaluator { - private final StringEvaluator subject; - private final StringEvaluator indexEvaluator; + private final Evaluator subject; + private final Evaluator indexEvaluator; - public LastIndexOfEvaluator(final StringEvaluator subject, final StringEvaluator indexEvaluator) { + public LastIndexOfEvaluator(final Evaluator subject, final Evaluator indexEvaluator) { this.subject = subject; this.indexEvaluator = indexEvaluator; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LengthEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LengthEvaluator.java index dec1f8f4b5..46bc24c091 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LengthEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LengthEvaluator.java @@ -22,20 +22,19 @@ import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult; import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; -import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; public class LengthEvaluator extends NumberEvaluator { - private final StringEvaluator subject; + private final Evaluator subject; - public LengthEvaluator(final StringEvaluator subject) { + public LengthEvaluator(final Evaluator subject) { this.subject = subject; } @Override public QueryResult evaluate(final Map attributes) { final String subjectValue = subject.evaluate(attributes).getValue(); - return new NumberQueryResult((long) ((subjectValue == null) ? 0 : subjectValue.length())); + return new NumberQueryResult((long) (subjectValue == null ? 0 : subjectValue.length())); } @Override diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanEvaluator.java index 4b1beac776..e9bf231dfe 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanEvaluator.java @@ -21,15 +21,14 @@ import java.util.Map; import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; -import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; public class LessThanEvaluator extends BooleanEvaluator { - private final NumberEvaluator subject; - private final NumberEvaluator comparison; + private final Evaluator subject; + private final Evaluator comparison; - public LessThanEvaluator(final NumberEvaluator subject, final NumberEvaluator comparison) { + public LessThanEvaluator(final Evaluator subject, final Evaluator comparison) { this.subject = subject; this.comparison = comparison; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanOrEqualEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanOrEqualEvaluator.java index a07e8be521..5bf5ee5ee4 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanOrEqualEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanOrEqualEvaluator.java @@ -21,15 +21,14 @@ import java.util.Map; import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; -import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; public class LessThanOrEqualEvaluator extends BooleanEvaluator { - private final NumberEvaluator subject; - private final NumberEvaluator comparison; + private final Evaluator subject; + private final Evaluator comparison; - public LessThanOrEqualEvaluator(final NumberEvaluator subject, final NumberEvaluator comparison) { + public LessThanOrEqualEvaluator(final Evaluator subject, final Evaluator comparison) { this.subject = subject; this.comparison = comparison; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MatchesEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MatchesEvaluator.java index 1bb86a5c34..6f017ea5b7 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MatchesEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MatchesEvaluator.java @@ -23,17 +23,16 @@ import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; -import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.literals.StringLiteralEvaluator; public class MatchesEvaluator extends BooleanEvaluator { - private final StringEvaluator subject; - private final StringEvaluator search; + private final Evaluator subject; + private final Evaluator search; private final Pattern compiledPattern; - public MatchesEvaluator(final StringEvaluator subject, final StringEvaluator search) { + public MatchesEvaluator(final Evaluator subject, final Evaluator search) { this.subject = subject; this.search = search; diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MinusEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MinusEvaluator.java index 066597966d..05551879bb 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MinusEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MinusEvaluator.java @@ -25,10 +25,10 @@ import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; public class MinusEvaluator extends NumberEvaluator { - private final NumberEvaluator subject; - private final NumberEvaluator minusValue; + private final Evaluator subject; + private final Evaluator minusValue; - public MinusEvaluator(final NumberEvaluator subject, final NumberEvaluator minusValue) { + public MinusEvaluator(final Evaluator subject, final Evaluator minusValue) { this.subject = subject; this.minusValue = minusValue; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ModEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ModEvaluator.java index 6b7b8388df..cf2121c0f5 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ModEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ModEvaluator.java @@ -25,10 +25,10 @@ import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; public class ModEvaluator extends NumberEvaluator { - private final NumberEvaluator subject; - private final NumberEvaluator modValue; + private final Evaluator subject; + private final Evaluator modValue; - public ModEvaluator(final NumberEvaluator subject, final NumberEvaluator modValue) { + public ModEvaluator(final Evaluator subject, final Evaluator modValue) { this.subject = subject; this.modValue = modValue; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MultiplyEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MultiplyEvaluator.java index a3b03ed94e..4480c98b84 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MultiplyEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MultiplyEvaluator.java @@ -25,10 +25,10 @@ import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; public class MultiplyEvaluator extends NumberEvaluator { - private final NumberEvaluator subject; - private final NumberEvaluator multiplyValue; + private final Evaluator subject; + private final Evaluator multiplyValue; - public MultiplyEvaluator(final NumberEvaluator subject, final NumberEvaluator multiplyValue) { + public MultiplyEvaluator(final Evaluator subject, final Evaluator multiplyValue) { this.subject = subject; this.multiplyValue = multiplyValue; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotEvaluator.java index 327e620890..2a09e332f8 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotEvaluator.java @@ -25,9 +25,9 @@ import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; public class NotEvaluator extends BooleanEvaluator { - private final BooleanEvaluator subjectEvaluator; + private final Evaluator subjectEvaluator; - public NotEvaluator(final BooleanEvaluator subjectEvaluator) { + public NotEvaluator(final Evaluator subjectEvaluator) { this.subjectEvaluator = subjectEvaluator; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NumberToDateEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NumberToDateEvaluator.java index 672d855dc9..6867202ad0 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NumberToDateEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NumberToDateEvaluator.java @@ -22,14 +22,13 @@ import java.util.Map; import org.apache.nifi.attribute.expression.language.evaluation.DateEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.DateQueryResult; import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; -import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; public class NumberToDateEvaluator extends DateEvaluator { - private final NumberEvaluator subject; + private final Evaluator subject; - public NumberToDateEvaluator(final NumberEvaluator subject) { + public NumberToDateEvaluator(final Evaluator subject) { this.subject = subject; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OrEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OrEvaluator.java index ab3a16ce1c..719fa11448 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OrEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OrEvaluator.java @@ -25,10 +25,10 @@ import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; public class OrEvaluator extends BooleanEvaluator { - private final BooleanEvaluator subjectEvaluator; - private final BooleanEvaluator rhsEvaluator; + private final Evaluator subjectEvaluator; + private final Evaluator rhsEvaluator; - public OrEvaluator(final BooleanEvaluator subjectEvaluator, final BooleanEvaluator rhsEvaluator) { + public OrEvaluator(final Evaluator subjectEvaluator, final Evaluator rhsEvaluator) { this.subjectEvaluator = subjectEvaluator; this.rhsEvaluator = rhsEvaluator; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.java index dff693dd0a..7c9278f5c2 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.java @@ -25,10 +25,10 @@ import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; public class PlusEvaluator extends NumberEvaluator { - private final NumberEvaluator subject; - private final NumberEvaluator plusValue; + private final Evaluator subject; + private final Evaluator plusValue; - public PlusEvaluator(final NumberEvaluator subject, final NumberEvaluator plusValue) { + public PlusEvaluator(final Evaluator subject, final Evaluator plusValue) { this.subject = subject; this.plusValue = plusValue; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PrependEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PrependEvaluator.java index 5b66b8fcaa..1053328f55 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PrependEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PrependEvaluator.java @@ -25,10 +25,10 @@ import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResul public class PrependEvaluator extends StringEvaluator { - private final StringEvaluator subject; - private final StringEvaluator prependEvaluator; + private final Evaluator subject; + private final Evaluator prependEvaluator; - public PrependEvaluator(final StringEvaluator subject, final StringEvaluator prepend) { + public PrependEvaluator(final Evaluator subject, final Evaluator prepend) { this.subject = subject; this.prependEvaluator = prepend; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceAllEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceAllEvaluator.java index 19ed63ef71..b7962336ba 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceAllEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceAllEvaluator.java @@ -25,11 +25,11 @@ import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResul public class ReplaceAllEvaluator extends StringEvaluator { - private final StringEvaluator subject; - private final StringEvaluator search; - private final StringEvaluator replacement; + private final Evaluator subject; + private final Evaluator search; + private final Evaluator replacement; - public ReplaceAllEvaluator(final StringEvaluator subject, final StringEvaluator search, final StringEvaluator replacement) { + public ReplaceAllEvaluator(final Evaluator subject, final Evaluator search, final Evaluator replacement) { this.subject = subject; this.search = search; this.replacement = replacement; diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEmptyEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEmptyEvaluator.java index fe08303e41..9c2cb6d31e 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEmptyEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEmptyEvaluator.java @@ -24,10 +24,10 @@ import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; public class ReplaceEmptyEvaluator extends StringEvaluator { - private final StringEvaluator subjectEvaluator; - private final StringEvaluator replacementEvaluator; + private final Evaluator subjectEvaluator; + private final Evaluator replacementEvaluator; - public ReplaceEmptyEvaluator(final StringEvaluator subjectEvaluator, final StringEvaluator replacementEvaluator) { + public ReplaceEmptyEvaluator(final Evaluator subjectEvaluator, final Evaluator replacementEvaluator) { this.subjectEvaluator = subjectEvaluator; this.replacementEvaluator = replacementEvaluator; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEvaluator.java index d6a7c5b7cc..e55e9630c6 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEvaluator.java @@ -25,11 +25,11 @@ import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResul public class ReplaceEvaluator extends StringEvaluator { - private final StringEvaluator subject; - private final StringEvaluator search; - private final StringEvaluator replacement; + private final Evaluator subject; + private final Evaluator search; + private final Evaluator replacement; - public ReplaceEvaluator(final StringEvaluator subject, final StringEvaluator search, final StringEvaluator replacement) { + public ReplaceEvaluator(final Evaluator subject, final Evaluator search, final Evaluator replacement) { this.subject = subject; this.search = search; this.replacement = replacement; diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceNullEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceNullEvaluator.java index 91333b727b..e8487c6bd4 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceNullEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceNullEvaluator.java @@ -25,10 +25,10 @@ import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResul public class ReplaceNullEvaluator extends StringEvaluator { - private final StringEvaluator subject; - private final StringEvaluator resultEvaluator; + private final Evaluator subject; + private final Evaluator resultEvaluator; - public ReplaceNullEvaluator(final StringEvaluator subject, final StringEvaluator resultEvaluator) { + public ReplaceNullEvaluator(final Evaluator subject, final Evaluator resultEvaluator) { this.subject = subject; this.resultEvaluator = resultEvaluator; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StartsWithEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StartsWithEvaluator.java index c6a7eb7c72..6a4355045d 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StartsWithEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StartsWithEvaluator.java @@ -22,14 +22,13 @@ import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; -import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; public class StartsWithEvaluator extends BooleanEvaluator { - private final StringEvaluator subject; - private final StringEvaluator search; + private final Evaluator subject; + private final Evaluator search; - public StartsWithEvaluator(final StringEvaluator subject, final StringEvaluator search) { + public StartsWithEvaluator(final Evaluator subject, final Evaluator search) { this.subject = subject; this.search = search; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StringToDateEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StringToDateEvaluator.java index 1575d75377..590176e096 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StringToDateEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StringToDateEvaluator.java @@ -26,15 +26,14 @@ import org.apache.nifi.attribute.expression.language.evaluation.DateEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.DateQueryResult; import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; -import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; import org.apache.nifi.attribute.expression.language.exception.IllegalAttributeException; public class StringToDateEvaluator extends DateEvaluator { - private final StringEvaluator subject; - private final StringEvaluator format; + private final Evaluator subject; + private final Evaluator format; - public StringToDateEvaluator(final StringEvaluator subject, final StringEvaluator format) { + public StringToDateEvaluator(final Evaluator subject, final Evaluator format) { this.subject = subject; this.format = format; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterEvaluator.java index a8b7357d94..6625cb6571 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterEvaluator.java @@ -25,10 +25,10 @@ import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResul public class SubstringAfterEvaluator extends StringEvaluator { - private final StringEvaluator subject; - private final StringEvaluator afterEvaluator; + private final Evaluator subject; + private final Evaluator afterEvaluator; - public SubstringAfterEvaluator(final StringEvaluator subject, final StringEvaluator afterEvaluator) { + public SubstringAfterEvaluator(final Evaluator subject, final Evaluator afterEvaluator) { this.subject = subject; this.afterEvaluator = afterEvaluator; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterLastEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterLastEvaluator.java index 3dac40ebe9..2cc9c3c346 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterLastEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterLastEvaluator.java @@ -25,10 +25,10 @@ import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResul public class SubstringAfterLastEvaluator extends StringEvaluator { - private final StringEvaluator subject; - private final StringEvaluator afterEvaluator; + private final Evaluator subject; + private final Evaluator afterEvaluator; - public SubstringAfterLastEvaluator(final StringEvaluator subject, final StringEvaluator afterEvaluator) { + public SubstringAfterLastEvaluator(final Evaluator subject, final Evaluator afterEvaluator) { this.subject = subject; this.afterEvaluator = afterEvaluator; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeEvaluator.java index 92b136cd83..4b8c3d0207 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeEvaluator.java @@ -25,10 +25,10 @@ import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResul public class SubstringBeforeEvaluator extends StringEvaluator { - private final StringEvaluator subject; - private final StringEvaluator beforeEvaluator; + private final Evaluator subject; + private final Evaluator beforeEvaluator; - public SubstringBeforeEvaluator(final StringEvaluator subject, final StringEvaluator beforeEvaluator) { + public SubstringBeforeEvaluator(final Evaluator subject, final Evaluator beforeEvaluator) { this.subject = subject; this.beforeEvaluator = beforeEvaluator; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeLastEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeLastEvaluator.java index 818a3dc9f0..14548c1229 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeLastEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeLastEvaluator.java @@ -25,10 +25,10 @@ import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResul public class SubstringBeforeLastEvaluator extends StringEvaluator { - private final StringEvaluator subject; - private final StringEvaluator beforeEvaluator; + private final Evaluator subject; + private final Evaluator beforeEvaluator; - public SubstringBeforeLastEvaluator(final StringEvaluator subject, final StringEvaluator beforeEvaluator) { + public SubstringBeforeLastEvaluator(final Evaluator subject, final Evaluator beforeEvaluator) { this.subject = subject; this.beforeEvaluator = beforeEvaluator; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringEvaluator.java index da11c45464..71b78f1182 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringEvaluator.java @@ -19,24 +19,23 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions; import java.util.Map; import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; -import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; public class SubstringEvaluator extends StringEvaluator { - private final StringEvaluator subject; - private final NumberEvaluator startIndex; - private final NumberEvaluator endIndex; + private final Evaluator subject; + private final Evaluator startIndex; + private final Evaluator endIndex; - public SubstringEvaluator(final StringEvaluator subject, final NumberEvaluator startIndex, final NumberEvaluator endIndex) { + public SubstringEvaluator(final Evaluator subject, final Evaluator startIndex, final Evaluator endIndex) { this.subject = subject; this.startIndex = startIndex; this.endIndex = endIndex; } - public SubstringEvaluator(final StringEvaluator subject, final NumberEvaluator startIndex) { + public SubstringEvaluator(final Evaluator subject, final Evaluator startIndex) { this.subject = subject; this.startIndex = startIndex; this.endIndex = null; diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToLowerEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToLowerEvaluator.java index 7de01eb2f3..33175f8b51 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToLowerEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToLowerEvaluator.java @@ -25,16 +25,16 @@ import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResul public class ToLowerEvaluator extends StringEvaluator { - private final StringEvaluator subject; + private final Evaluator subject; - public ToLowerEvaluator(final StringEvaluator subject) { + public ToLowerEvaluator(final Evaluator subject) { this.subject = subject; } @Override public QueryResult evaluate(final Map attributes) { final String subjectValue = subject.evaluate(attributes).getValue(); - return new StringQueryResult((subjectValue == null) ? null : subjectValue.toLowerCase()); + return new StringQueryResult(subjectValue == null ? null : subjectValue.toLowerCase()); } @Override diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToNumberEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToNumberEvaluator.java index 1d4664acfe..736f0b9f15 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToNumberEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToNumberEvaluator.java @@ -22,20 +22,19 @@ import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult; import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; -import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; public class ToNumberEvaluator extends NumberEvaluator { - private final StringEvaluator subject; + private final Evaluator subject; - public ToNumberEvaluator(final StringEvaluator subject) { + public ToNumberEvaluator(final Evaluator subject) { this.subject = subject; } @Override public QueryResult evaluate(final Map attributes) { final String subjectValue = subject.evaluate(attributes).getValue(); - return new NumberQueryResult((subjectValue == null || subjectValue.trim().isEmpty()) ? null : Long.valueOf(subjectValue)); + return new NumberQueryResult(subjectValue == null || subjectValue.trim().isEmpty() ? null : Long.valueOf(subjectValue)); } @Override diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToRadixEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToRadixEvaluator.java index aab094dcc5..1cf7e554ac 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToRadixEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToRadixEvaluator.java @@ -20,22 +20,21 @@ import java.util.Arrays; import java.util.Map; import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; -import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; public class ToRadixEvaluator extends StringEvaluator { - private final NumberEvaluator numberEvaluator; - private final NumberEvaluator radixEvaluator; - private final NumberEvaluator minimumWidthEvaluator; + private final Evaluator numberEvaluator; + private final Evaluator radixEvaluator; + private final Evaluator minimumWidthEvaluator; - public ToRadixEvaluator(final NumberEvaluator subject, final NumberEvaluator radixEvaluator) { + public ToRadixEvaluator(final Evaluator subject, final Evaluator radixEvaluator) { this(subject, radixEvaluator, null); } - public ToRadixEvaluator(final NumberEvaluator subject, final NumberEvaluator radixEvaluator, final NumberEvaluator minimumWidthEvaluator) { + public ToRadixEvaluator(final Evaluator subject, final Evaluator radixEvaluator, final Evaluator minimumWidthEvaluator) { this.numberEvaluator = subject; this.radixEvaluator = radixEvaluator; this.minimumWidthEvaluator = minimumWidthEvaluator; diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToUpperEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToUpperEvaluator.java index 29bc61d9a4..f3174c2ef4 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToUpperEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToUpperEvaluator.java @@ -25,16 +25,16 @@ import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResul public class ToUpperEvaluator extends StringEvaluator { - private final StringEvaluator subject; + private final Evaluator subject; - public ToUpperEvaluator(final StringEvaluator subject) { + public ToUpperEvaluator(final Evaluator subject) { this.subject = subject; } @Override public QueryResult evaluate(final Map attributes) { final String subjectValue = subject.evaluate(attributes).getValue(); - return new StringQueryResult((subjectValue == null) ? null : subjectValue.toUpperCase()); + return new StringQueryResult(subjectValue == null ? null : subjectValue.toUpperCase()); } @Override diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/TrimEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/TrimEvaluator.java index b9926d7e34..e44661f164 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/TrimEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/TrimEvaluator.java @@ -25,9 +25,9 @@ import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResul public class TrimEvaluator extends StringEvaluator { - private final StringEvaluator subject; + private final Evaluator subject; - public TrimEvaluator(final StringEvaluator subject) { + public TrimEvaluator(final Evaluator subject) { this.subject = subject; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlDecodeEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlDecodeEvaluator.java index 6c3ba14bc6..c61d55dd8a 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlDecodeEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlDecodeEvaluator.java @@ -27,9 +27,9 @@ import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResul public class UrlDecodeEvaluator extends StringEvaluator { - private final StringEvaluator subject; + private final Evaluator subject; - public UrlDecodeEvaluator(final StringEvaluator subject) { + public UrlDecodeEvaluator(final Evaluator subject) { this.subject = subject; } @@ -42,7 +42,7 @@ public class UrlDecodeEvaluator extends StringEvaluator { try { return new StringQueryResult(URLDecoder.decode(subjectValue, "UTF-8")); - } catch (UnsupportedEncodingException e) { + } catch (final UnsupportedEncodingException e) { return null; // won't happen. It's UTF-8 } } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlEncodeEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlEncodeEvaluator.java index d1e2caa5b6..769f6df32e 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlEncodeEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlEncodeEvaluator.java @@ -27,9 +27,9 @@ import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResul public class UrlEncodeEvaluator extends StringEvaluator { - private final StringEvaluator subject; + private final Evaluator subject; - public UrlEncodeEvaluator(final StringEvaluator subject) { + public UrlEncodeEvaluator(final Evaluator subject) { this.subject = subject; } @@ -42,7 +42,7 @@ public class UrlEncodeEvaluator extends StringEvaluator { try { return new StringQueryResult(URLEncoder.encode(subjectValue, "UTF-8")); - } catch (UnsupportedEncodingException e) { + } catch (final UnsupportedEncodingException e) { return null; // won't happen. It's UTF-8 } } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/ToLiteralEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/ToLiteralEvaluator.java new file mode 100644 index 0000000000..e297eeae1c --- /dev/null +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/ToLiteralEvaluator.java @@ -0,0 +1,43 @@ +/* + * 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.nifi.attribute.expression.language.evaluation.literals; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class ToLiteralEvaluator extends StringEvaluator { + private final Evaluator argEvaluator; + + public ToLiteralEvaluator(final Evaluator argEvaluator) { + this.argEvaluator = argEvaluator; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final Object result = argEvaluator.evaluate(attributes); + return new StringQueryResult(result == null ? null : result.toString()); + } + + @Override + public Evaluator getSubjectEvaluator() { + return null; + } +} diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/JoinEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/JoinEvaluator.java index 81c325da9e..f80924bcdd 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/JoinEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/JoinEvaluator.java @@ -25,13 +25,13 @@ import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResul public class JoinEvaluator extends StringEvaluator implements ReduceEvaluator { - private final StringEvaluator subjectEvaluator; - private final StringEvaluator delimiterEvaluator; + private final Evaluator subjectEvaluator; + private final Evaluator delimiterEvaluator; private final StringBuilder sb = new StringBuilder(); private int evalCount = 0; - public JoinEvaluator(final StringEvaluator subject, final StringEvaluator delimiter) { + public JoinEvaluator(final Evaluator subject, final Evaluator delimiter) { this.subjectEvaluator = subject; this.delimiterEvaluator = delimiter; } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java index d9dd4d3a1b..ccf9d1908a 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java @@ -23,7 +23,7 @@ import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResu import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; -public class AllAttributesEvaluator extends BooleanEvaluator { +public class AllAttributesEvaluator extends BooleanEvaluator implements IteratingEvaluator { private final BooleanEvaluator booleanEvaluator; private final MultiAttributeEvaluator multiAttributeEvaluator; @@ -65,4 +65,9 @@ public class AllAttributesEvaluator extends BooleanEvaluator { public Evaluator getSubjectEvaluator() { return null; } + + @Override + public Evaluator getLogicEvaluator() { + return booleanEvaluator; + } } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java index 9192958164..6e8d485310 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java @@ -23,7 +23,7 @@ import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResu import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; -public class AnyAttributeEvaluator extends BooleanEvaluator { +public class AnyAttributeEvaluator extends BooleanEvaluator implements IteratingEvaluator { private final BooleanEvaluator booleanEvaluator; private final MultiAttributeEvaluator multiAttributeEvaluator; @@ -65,4 +65,9 @@ public class AnyAttributeEvaluator extends BooleanEvaluator { public Evaluator getSubjectEvaluator() { return null; } + + @Override + public Evaluator getLogicEvaluator() { + return booleanEvaluator; + } } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/DelineatedAttributeEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/DelineatedAttributeEvaluator.java index 209c86f3a5..e39d01cb49 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/DelineatedAttributeEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/DelineatedAttributeEvaluator.java @@ -20,19 +20,18 @@ import java.util.Map; import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; -import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; public class DelineatedAttributeEvaluator extends MultiAttributeEvaluator { - private final StringEvaluator subjectEvaluator; - private final StringEvaluator delimiterEvaluator; + private final Evaluator subjectEvaluator; + private final Evaluator delimiterEvaluator; private final int evaluationType; private String[] delineatedValues; private int evaluationCount = 0; private int evaluationsLeft = 1; - public DelineatedAttributeEvaluator(final StringEvaluator subjectEvaluator, final StringEvaluator delimiterEvaluator, final int evaluationType) { + public DelineatedAttributeEvaluator(final Evaluator subjectEvaluator, final Evaluator delimiterEvaluator, final int evaluationType) { this.subjectEvaluator = subjectEvaluator; this.delimiterEvaluator = delimiterEvaluator; this.evaluationType = evaluationType; @@ -66,6 +65,11 @@ public class DelineatedAttributeEvaluator extends MultiAttributeEvaluator { return new StringQueryResult(delineatedValues[evaluationCount++]); } + @Override + public Evaluator getLogicEvaluator() { + return subjectEvaluator; + } + @Override public int getEvaluationsRemaining() { return evaluationsLeft; diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyMatchingAttributeEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/IteratingEvaluator.java similarity index 67% rename from nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyMatchingAttributeEvaluator.java rename to nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/IteratingEvaluator.java index 8c07278c3a..5156345fc9 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyMatchingAttributeEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/IteratingEvaluator.java @@ -1,21 +1,33 @@ -/* - * 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.nifi.attribute.expression.language.evaluation.selection; - -public class AnyMatchingAttributeEvaluator { - -} +/* + * 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.nifi.attribute.expression.language.evaluation.selection; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; + +/** + * Interface for an Evaluator that should be evaluated multiple times + * + * @param return type of evaluator + */ +public interface IteratingEvaluator extends Evaluator { + + /** + * @return the evaluator that evaluates some sort of logic against its subject + */ + Evaluator getLogicEvaluator(); + +} diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MappingEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MappingEvaluator.java index 2b8c488b43..e007a56392 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MappingEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MappingEvaluator.java @@ -24,9 +24,9 @@ import org.apache.nifi.attribute.expression.language.evaluation.reduce.ReduceEva import org.apache.nifi.expression.AttributeExpression.ResultType; public class MappingEvaluator implements Evaluator { - private final ReduceEvaluator mappingEvaluator; private final MultiAttributeEvaluator multiAttributeEvaluator; + private String token; public MappingEvaluator(final ReduceEvaluator mappingEvaluator, final MultiAttributeEvaluator multiAttributeEval) { this.mappingEvaluator = mappingEvaluator; @@ -59,4 +59,13 @@ public class MappingEvaluator implements Evaluator { return null; } + @Override + public String getToken() { + return token; + } + + @Override + public void setToken(final String token) { + this.token = token; + } } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiAttributeEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiAttributeEvaluator.java index f80ed972b9..78b5af4a63 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiAttributeEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiAttributeEvaluator.java @@ -18,7 +18,6 @@ package org.apache.nifi.attribute.expression.language.evaluation.selection; import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; -public abstract class MultiAttributeEvaluator extends StringEvaluator { - +public abstract class MultiAttributeEvaluator extends StringEvaluator implements IteratingEvaluator { public abstract int getEvaluationType(); } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiMatchAttributeEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiMatchAttributeEvaluator.java index 1d0be8bf61..8292f84153 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiMatchAttributeEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiMatchAttributeEvaluator.java @@ -79,4 +79,9 @@ public class MultiMatchAttributeEvaluator extends MultiAttributeEvaluator { public int getEvaluationType() { return evaluationType; } + + @Override + public Evaluator getLogicEvaluator() { + return this; + } } diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java index 6dabc0abe3..509d7dd2c7 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java @@ -61,4 +61,9 @@ public class MultiNamedAttributeEvaluator extends MultiAttributeEvaluator { public int getEvaluationType() { return evaluationType; } + + @Override + public Evaluator getLogicEvaluator() { + return this; + } } diff --git a/nifi/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java b/nifi/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java index f34326187b..131bcde9e6 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java +++ b/nifi/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java @@ -17,10 +17,12 @@ package org.apache.nifi.attribute.expression.language; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.text.SimpleDateFormat; import java.util.Calendar; +import java.util.Collections; import java.util.Date; import java.util.HashMap; import java.util.HashSet; @@ -52,6 +54,8 @@ public class TestQuery { assertValid("${now():format('yyyy/MM/dd')}"); assertInvalid("${attr:times(3)}"); assertValid("${attr:toNumber():multiply(3)}"); + assertValid("${hostname()}"); + assertValid("${literal(3)}"); // left here because it's convenient for looking at the output //System.out.println(Query.compile("").evaluate(null)); } @@ -296,7 +300,7 @@ public class TestQuery { } private String evaluateQueryForEscape(final String queryString, final Map attributes) { - FlowFile mockFlowFile = Mockito.mock(FlowFile.class); + final FlowFile mockFlowFile = Mockito.mock(FlowFile.class); Mockito.when(mockFlowFile.getAttributes()).thenReturn(attributes); Mockito.when(mockFlowFile.getId()).thenReturn(1L); Mockito.when(mockFlowFile.getEntryDate()).thenReturn(System.currentTimeMillis()); @@ -429,7 +433,7 @@ public class TestQuery { @Test public void testExtractExpressionTypes() { - List types = Query.extractResultTypes("${hello:equals( ${goodbye} )} or just hi, ${bob}, are you ${bob.age:toNumber()} yet? $$$${bob}"); + final List types = Query.extractResultTypes("${hello:equals( ${goodbye} )} or just hi, ${bob}, are you ${bob.age:toNumber()} yet? $$$${bob}"); assertEquals(3, types.size()); assertEquals(ResultType.BOOLEAN, types.get(0)); assertEquals(ResultType.STRING, types.get(1)); @@ -493,7 +497,7 @@ public class TestQuery { final Map attributes = new HashMap<>(); attributes.put("xx", "say 'hi'"); - String query = "${xx:replace( \"'hi'\", '\\\"hello\\\"' )}"; + final String query = "${xx:replace( \"'hi'\", '\\\"hello\\\"' )}"; System.out.println(query); verifyEquals(query, attributes, "say \"hello\""); } @@ -1112,12 +1116,52 @@ public class TestQuery { assertEquals("{ xyz }", Query.evaluateExpressions(query, attributes)); } + @Test + public void testLiteralFunction() { + final Map attrs = Collections. emptyMap(); + verifyEquals("${literal(2):gt(1)}", attrs, true); + verifyEquals("${literal('hello'):substring(0, 1):equals('h')}", attrs, true); + } + + @Test + public void testFunctionAfterReduce() { + // Cannot call gt(2) after count() because count() is a 'reducing function' + // and must be the last function in an expression. + assertFalse(Query.isValidExpression("${allMatchingAttributes('a.*'):contains('2'):count():gt(2)}")); + + final Map attributes = new HashMap<>(); + attributes.put("a.1", "245"); + attributes.put("a.2", "123"); + attributes.put("a.3", "732"); + attributes.put("a.4", "343"); + attributes.put("a.5", "553"); + + final String endsWithCount = "${allMatchingAttributes('a.*'):contains('2'):count()}"; + assertTrue(Query.isValidExpression(endsWithCount)); + verifyEquals(endsWithCount, attributes, 3L); + + // in order to check if value is greater than 2, need to first evaluate the + // 'aggregate' and 'reducing' functions as an inner expression. Then we can + // use the literal() function to make the result of the inner expression the subject + // of the function gt() + final String usingLiteral = "${literal(" + endsWithCount + "):gt(2)}"; + assertTrue(Query.isValidExpression(usingLiteral)); + verifyEquals(usingLiteral, attributes, true); + + attributes.clear(); + attributes.put("a1", "123"); + attributes.put("a2", "321"); + verifyEquals("${allMatchingAttributes('a.*'):contains('2')}", attributes, true); + verifyEquals("${allMatchingAttributes('a.*'):contains('2'):toUpper():equals('TRUE')}", attributes, true); + verifyEquals("${allMatchingAttributes('a.*'):contains('2'):equals('true'):and( ${literal(true)} )}", attributes, true); + } + private void verifyEquals(final String expression, final Map attributes, final Object expectedResult) { Query.validateExpression(expression, false); assertEquals(String.valueOf(expectedResult), Query.evaluateExpressions(expression, attributes, null)); - Query query = Query.compile(expression); - QueryResult result = query.evaluate(attributes); + final Query query = Query.compile(expression); + final QueryResult result = query.evaluate(attributes); if (expectedResult instanceof Number) { assertEquals(ResultType.NUMBER, result.getResultType()); diff --git a/nifi/nifi-docs/src/main/asciidoc/expression-language-guide.adoc b/nifi/nifi-docs/src/main/asciidoc/expression-language-guide.adoc index 90388094d5..d8ccdb4cac 100644 --- a/nifi/nifi-docs/src/main/asciidoc/expression-language-guide.adoc +++ b/nifi/nifi-docs/src/main/asciidoc/expression-language-guide.adoc @@ -1317,9 +1317,9 @@ Expressions will provide the following results: and Midnight GMT on January 1, 2014. The Expression `${time:toDate("yyyy/MM/dd HH:mm:ss.SSS'Z'")}` will result in the number of milliseconds between Midnight GMT on January 1, 1970 and 15:36:03.264 GMT on December 31, 2014. - Often, this function is used in conjunction with the <> function to change the format of a date/time. For example, - if the attribute "date" has the value "12-24-2014" and we want to change the format to "2014/12/24", we can do so by - chaining together the two functions: `${date:toDate('MM-dd-yyyy'):format('yyyy/MM/dd')}`. +Often, this function is used in conjunction with the <> function to change the format of a date/time. For example, +if the attribute "date" has the value "12-24-2014" and we want to change the format to "2014/12/24", we can do so by +chaining together the two functions: `${date:toDate('MM-dd-yyyy'):format('yyyy/MM/dd')}`. @@ -1404,7 +1404,7 @@ an error when validating the function. *Return Type*: [.returnType]#String# -*Examples*: The IP address of the machine can be obtained by using the Expresison `${ip()}`. +*Examples*: The IP address of the machine can be obtained by using the Expression `${ip()}`. @@ -1471,8 +1471,26 @@ an error when validating the function. +[.function] +=== literal +*Description*: [.description]#Returns its argument as a literal String value. This is useful in order to treat a string or a number + at the beginning of an Expression as an actual value, rather than treating it as an attribute name. Additionally, it + can be used when the argument is an embedded Expression that we would then like to evaluate additional functions against.# + +*Subject Type*: [.subjectless]#No Subject# +*Arguments*: + + - [.argName]#_value_# : [.argDesc]#The value to be treated as a literal string, number, or boolean value.# + +*Return Type*: [.returnType]#String# + +*Examples*: `${literal(2):gt(1)}` returns `true` + +`${literal( ${allMatchingAttributes('a.*'):count()} ):gt(3)}` returns true if there are more than 3 attributes whose +names begin with the letter `a`. + [[multi]] == Evaluating Multiple Attributes diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java index a24f4d00e2..80b179542d 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java @@ -89,7 +89,7 @@ import org.xml.sax.InputSource; + "Name into which the result will be placed (if the Destination is flowfile-attribute; otherwise, the property name is ignored). " + "The value of the property must be a valid XPath expression. If the XPath evaluates to more than one node and the Return Type is " + "set to 'nodeset' (either directly, or via 'auto-detect' with a Destination of " - + "'flowfile-content', the FlowFile will be unmodified and will be routed to failure. If the XPath does not " + + "'flowfile-content'), the FlowFile will be unmodified and will be routed to failure. If the XPath does not " + "evaluate to a Node, the FlowFile will be routed to 'unmatched' without having its contents modified. If Destination is " + "flowfile-attribute and the expression matches nothing, attributes will be created with empty strings as the value, and the " + "FlowFile will always be routed to 'matched'") diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java index 1654a4f602..0001bb3caf 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java @@ -67,9 +67,10 @@ import org.apache.http.conn.ssl.TrustSelfSignedStrategy; import org.apache.http.impl.client.BasicCredentialsProvider; import org.apache.http.impl.client.HttpClientBuilder; import org.apache.http.impl.conn.BasicHttpClientConnectionManager; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.behavior.WritesAttribute; import org.apache.nifi.annotation.lifecycle.OnShutdown; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; @@ -91,7 +92,10 @@ import org.apache.nifi.util.StopWatch; @Tags({"get", "fetch", "poll", "http", "https", "ingest", "source", "input"}) @CapabilityDescription("Fetches a file via HTTP") -@WritesAttribute(attribute = "filename", description = "the filename is set to the name of the file on the remote server") +@WritesAttributes({ + @WritesAttribute(attribute = "filename", description = "The filename is set to the name of the file on the remote server"), + @WritesAttribute(attribute = "mime.type", description = "The MIME Type of the FlowFile, as reported by the HTTP Content-Type header") +}) public class GetHTTP extends AbstractSessionFactoryProcessor { static final int PERSISTENCE_INTERVAL_MSEC = 10000; @@ -413,6 +417,15 @@ public class GetHTTP extends AbstractSessionFactoryProcessor { flowFile = session.putAttribute(flowFile, CoreAttributes.FILENAME.key(), context.getProperty(FILENAME).getValue()); flowFile = session.putAttribute(flowFile, this.getClass().getSimpleName().toLowerCase() + ".remote.source", source); flowFile = session.importFrom(response.getEntity().getContent(), flowFile); + + final Header contentTypeHeader = response.getFirstHeader("Content-Type"); + if (contentTypeHeader != null) { + final String contentType = contentTypeHeader.getValue(); + if (!contentType.trim().isEmpty()) { + flowFile = session.putAttribute(flowFile, CoreAttributes.MIME_TYPE.key(), contentType.trim()); + } + } + final long flowFileSize = flowFile.getSize(); stopWatch.stop(); final String dataRate = stopWatch.calculateDataRate(flowFileSize);