diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNode.java b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNode.java index c50cdf5840..56e865f1f9 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNode.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNode.java @@ -22,6 +22,7 @@ public interface LineageNode { * @return the identifier of the Clustered NiFi Node that generated the * event */ + @Deprecated String getClusterNodeIdentifier(); /** diff --git a/nifi/nifi-assembly/pom.xml b/nifi/nifi-assembly/pom.xml index 02a16f9384..9b17617b2a 100644 --- a/nifi/nifi-assembly/pom.xml +++ b/nifi/nifi-assembly/pom.xml @@ -276,6 +276,7 @@ language governing permissions and limitations under the License. --> 500 MB false 16 + 65536 100000 diff --git a/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EventNode.java b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EventNode.java index 12d9a4ff5e..1f8d1dcd46 100644 --- a/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EventNode.java +++ b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EventNode.java @@ -36,11 +36,13 @@ public class EventNode implements ProvenanceEventLineageNode { return String.valueOf(getEventIdentifier()); } + @Deprecated @Override public String getClusterNodeIdentifier() { return clusterNodeIdentifier; } + @Deprecated public void setClusterNodeIdentifier(final String nodeIdentifier) { this.clusterNodeIdentifier = nodeIdentifier; } diff --git a/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileNode.java b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileNode.java index fdc7470d99..94e7661aed 100644 --- a/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileNode.java +++ b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileNode.java @@ -39,6 +39,7 @@ public class FlowFileNode implements LineageNode { return creationTime; } + @Deprecated @Override public String getClusterNodeIdentifier() { return clusterNodeIdentifier; 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-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java b/nifi/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java index 43f02cab77..e25f5d6129 100644 --- a/nifi/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java +++ b/nifi/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java @@ -233,8 +233,7 @@ public class NiFiProperties extends Properties { * obtained. * * @return the NiFiProperties object to use - * @throws RuntimeException - * if unable to load properties file + * @throws RuntimeException if unable to load properties file */ public static synchronized NiFiProperties getInstance() { if (null == instance) { @@ -794,7 +793,7 @@ public class NiFiProperties extends Properties { final String scheme = (rawScheme == null) ? "http" : rawScheme; final String host; - final int port; + final Integer port; if ("http".equalsIgnoreCase(scheme)) { // get host if (StringUtils.isBlank(getProperty(WEB_HTTP_HOST))) { @@ -804,6 +803,10 @@ public class NiFiProperties extends Properties { } // get port port = getPort(); + + if (port == null) { + throw new RuntimeException(String.format("The %s must be specified if running in a cluster with %s set to false.", WEB_HTTP_PORT, CLUSTER_PROTOCOL_IS_SECURE)); + } } else { // get host if (StringUtils.isBlank(getProperty(WEB_HTTPS_HOST))) { @@ -813,6 +816,10 @@ public class NiFiProperties extends Properties { } // get port port = getSslPort(); + + if (port == null) { + throw new RuntimeException(String.format("The %s must be specified if running in a cluster with %s set to true.", WEB_HTTPS_PORT, CLUSTER_PROTOCOL_IS_SECURE)); + } } return InetSocketAddress.createUnresolved(host, port); @@ -824,8 +831,7 @@ public class NiFiProperties extends Properties { * configured. No directories will be created as a result of this operation. * * @return database repository path - * @throws InvalidPathException - * If the configured path is invalid + * @throws InvalidPathException If the configured path is invalid */ public Path getDatabaseRepositoryPath() { return Paths.get(getProperty(REPOSITORY_DATABASE_DIRECTORY)); @@ -836,8 +842,7 @@ public class NiFiProperties extends Properties { * configured. No directories will be created as a result of this operation. * * @return database repository path - * @throws InvalidPathException - * If the configured path is invalid + * @throws InvalidPathException If the configured path is invalid */ public Path getFlowFileRepositoryPath() { return Paths.get(getProperty(FLOWFILE_REPOSITORY_DIRECTORY)); @@ -850,8 +855,7 @@ public class NiFiProperties extends Properties { * operation. * * @return file repositories paths - * @throws InvalidPathException - * If any of the configured paths are invalid + * @throws InvalidPathException If any of the configured paths are invalid */ public Map getContentRepositoryPaths() { final Map contentRepositoryPaths = new HashMap<>(); diff --git a/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc index 039e5b9e8d..5535c35544 100644 --- a/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc +++ b/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc @@ -210,7 +210,7 @@ Here is an example entry using the name John Smith: ---- - + 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-external/nifi-example-bundle/nifi-nifi-example-nar/pom.xml b/nifi/nifi-external/nifi-example-bundle/nifi-nifi-example-nar/pom.xml new file mode 100644 index 0000000000..55df946bef --- /dev/null +++ b/nifi/nifi-external/nifi-example-bundle/nifi-nifi-example-nar/pom.xml @@ -0,0 +1,36 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-example-bundle + 0.1.1-incubating-SNAPSHOT + + + nifi-nifi-example-nar + nar + + + + org.apache.nifi + nifi-nifi-example-processors + ${project.version} + + + + diff --git a/nifi/nifi-external/nifi-example-bundle/nifi-nifi-example-processors/pom.xml b/nifi/nifi-external/nifi-example-bundle/nifi-nifi-example-processors/pom.xml new file mode 100644 index 0000000000..b4fbc85172 --- /dev/null +++ b/nifi/nifi-external/nifi-example-bundle/nifi-nifi-example-processors/pom.xml @@ -0,0 +1,70 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-example-bundle + 0.1.1-incubating-SNAPSHOT + + + nifi-nifi-example-processors + jar + + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-processor-utils + + + org.apache.nifi + nifi-mock + test + + + org.slf4j + slf4j-simple + test + + + commons-io + commons-io + + + junit + junit + test + + + + + + org.apache.rat + apache-rat-plugin + + + src/main/resources/file.txt + + + + + + diff --git a/nifi/nifi-external/nifi-example-bundle/nifi-nifi-example-processors/src/main/java/org/apache/nifi/processors/WriteResourceToStream.java b/nifi/nifi-external/nifi-example-bundle/nifi-nifi-example-processors/src/main/java/org/apache/nifi/processors/WriteResourceToStream.java new file mode 100644 index 0000000000..5d595b4417 --- /dev/null +++ b/nifi/nifi-external/nifi-example-bundle/nifi-nifi-example-processors/src/main/java/org/apache/nifi/processors/WriteResourceToStream.java @@ -0,0 +1,102 @@ +/* + * 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.processors; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +import org.apache.commons.io.IOUtils; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.ProcessorInitializationContext; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.io.OutputStreamCallback; + +@Tags({ "example", "resources" }) +@CapabilityDescription("This example processor loads a resource from the nar and writes it to the FlowFile content") +public class WriteResourceToStream extends AbstractProcessor { + + public static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("files that were successfully processed").build(); + public static final Relationship REL_FAILURE = new Relationship.Builder() + .name("failure") + .description("files that were not successfully processed").build(); + + private Set relationships; + + private String resourceData; + + @Override + protected void init(final ProcessorInitializationContext context) { + + final Set relationships = new HashSet(); + relationships.add(REL_SUCCESS); + relationships.add(REL_FAILURE); + this.relationships = Collections.unmodifiableSet(relationships); + + try { + this.resourceData = IOUtils.toString(Thread.currentThread() + .getContextClassLoader().getResourceAsStream("file.txt")); + } catch (IOException e) { + throw new RuntimeException("Unable to load resources", e); + } + } + + @Override + public Set getRelationships() { + return this.relationships; + } + + @OnScheduled + public void onScheduled(final ProcessContext context) { + + } + + @Override + public void onTrigger(final ProcessContext context, + final ProcessSession session) throws ProcessException { + FlowFile flowFile = session.get(); + if (flowFile == null) { + return; + } + + try { + flowFile = session.write(flowFile, new OutputStreamCallback() { + + @Override + public void process(OutputStream out) throws IOException { + IOUtils.write(resourceData, out); + + } + }); + session.transfer(flowFile, REL_SUCCESS); + } catch (ProcessException ex) { + getLogger().error("Unable to process", ex); + session.transfer(flowFile, REL_FAILURE); + } + } +} diff --git a/nifi/nifi-external/nifi-example-bundle/nifi-nifi-example-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi/nifi-external/nifi-example-bundle/nifi-nifi-example-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor new file mode 100644 index 0000000000..94485417f9 --- /dev/null +++ b/nifi/nifi-external/nifi-example-bundle/nifi-nifi-example-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor @@ -0,0 +1,15 @@ +# 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. +org.apache.nifi.processors.WriteResourceToStream \ No newline at end of file diff --git a/nifi/nifi-external/nifi-example-bundle/nifi-nifi-example-processors/src/main/resources/file.txt b/nifi/nifi-external/nifi-example-bundle/nifi-nifi-example-processors/src/main/resources/file.txt new file mode 100644 index 0000000000..f0e1d6c9e3 --- /dev/null +++ b/nifi/nifi-external/nifi-example-bundle/nifi-nifi-example-processors/src/main/resources/file.txt @@ -0,0 +1 @@ +this came from a resource \ No newline at end of file diff --git a/nifi/nifi-external/nifi-example-bundle/nifi-nifi-example-processors/src/test/java/org/apache/nifi/processors/WriteResourceToStreamTest.java b/nifi/nifi-external/nifi-example-bundle/nifi-nifi-example-processors/src/test/java/org/apache/nifi/processors/WriteResourceToStreamTest.java new file mode 100644 index 0000000000..7a84df58a2 --- /dev/null +++ b/nifi/nifi-external/nifi-example-bundle/nifi-nifi-example-processors/src/test/java/org/apache/nifi/processors/WriteResourceToStreamTest.java @@ -0,0 +1,47 @@ +/* + * 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.processors; + +import org.junit.Assert; + +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Before; +import org.junit.Test; + +public class WriteResourceToStreamTest { + + private TestRunner testRunner; + + @Before + public void init() { + testRunner = TestRunners.newTestRunner(WriteResourceToStream.class); + } + + @Test + public void testProcessor() { + testRunner.enqueue(new byte[] { 1, 2, 3, 4, 5 }); + testRunner.run(); + testRunner.assertAllFlowFilesTransferred(WriteResourceToStream.REL_SUCCESS, 1); + final byte[] data = testRunner + .getFlowFilesForRelationship(WriteResourceToStream.REL_SUCCESS).get(0) + .toByteArray(); + final String stringData = new String(data); + Assert.assertEquals("this came from a resource", stringData); + } + +} diff --git a/nifi/nifi-external/nifi-example-bundle/pom.xml b/nifi/nifi-external/nifi-example-bundle/pom.xml new file mode 100644 index 0000000000..542a6737eb --- /dev/null +++ b/nifi/nifi-external/nifi-example-bundle/pom.xml @@ -0,0 +1,33 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-external + 0.1.1-incubating-SNAPSHOT + + + nifi-example-bundle + pom + + + nifi-nifi-example-processors + nifi-nifi-example-nar + + + diff --git a/nifi/nifi-external/pom.xml b/nifi/nifi-external/pom.xml index 0c70c4ac60..8a2e585c8b 100644 --- a/nifi/nifi-external/pom.xml +++ b/nifi/nifi-external/pom.xml @@ -26,5 +26,7 @@ nifi-spark-receiver nifi-storm-spout + nifi-example-bundle + diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java index 61af49da42..742f03be9c 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java @@ -22,14 +22,21 @@ import java.util.Map; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.ControllerServiceLookup; public class MockConfigurationContext implements ConfigurationContext { private final Map properties; private final ControllerServiceLookup serviceLookup; + private final ControllerService service; public MockConfigurationContext(final Map properties, final ControllerServiceLookup serviceLookup) { + this(null, properties, serviceLookup); + } + + public MockConfigurationContext(final ControllerService service, final Map properties, final ControllerServiceLookup serviceLookup) { + this.service = service; this.properties = properties; this.serviceLookup = serviceLookup; } @@ -38,7 +45,7 @@ public class MockConfigurationContext implements ConfigurationContext { public PropertyValue getProperty(final PropertyDescriptor property) { String value = properties.get(property); if (value == null) { - value = property.getDefaultValue(); + value = getActualDescriptor(property).getDefaultValue(); } return new MockPropertyValue(value, serviceLookup); } @@ -47,4 +54,13 @@ public class MockConfigurationContext implements ConfigurationContext { public Map getProperties() { return new HashMap<>(this.properties); } + + private PropertyDescriptor getActualDescriptor(final PropertyDescriptor property) { + if (service == null) { + return property; + } + + final PropertyDescriptor resolved = service.getPropertyDescriptor(property.getName()); + return resolved == null ? property : resolved; + } } diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java index 1505899d71..048e2b92c3 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java @@ -214,8 +214,8 @@ public class StandardProcessorTestRunner implements TestRunner { executorService.shutdown(); try { - executorService.awaitTermination(runWait, TimeUnit.SECONDS); - } catch (InterruptedException e1) { + executorService.awaitTermination(runWait, TimeUnit.MILLISECONDS); + } catch (final InterruptedException e1) { } int finishedCount = 0; @@ -609,7 +609,7 @@ public class StandardProcessorTestRunner implements TestRunner { } try { - final ConfigurationContext configContext = new MockConfigurationContext(configuration.getProperties(), context); + final ConfigurationContext configContext = new MockConfigurationContext(service, configuration.getProperties(), context); ReflectionUtils.invokeMethodsWithAnnotation(OnEnabled.class, service, configContext); } catch (final InvocationTargetException ite) { ite.getCause().printStackTrace(); diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java index f0fbea8ca6..fb9fc78600 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java @@ -165,7 +165,7 @@ public interface TestRunner { * @param stopOnFinish whether or not to run the Processor methods that are * annotated with {@link nifi.processor.annotation.OnStopped @OnStopped} * @param initialize true if must initialize - * @param runWait indiciates the amount of time in milliseconds that the framework should wait for + * @param runWait indicates the amount of time in milliseconds that the framework should wait for * processors to stop running before calling the {@link nifi.processor.annotation.OnUnscheduled @OnUnscheduled} annotation */ void run(int iterations, boolean stopOnFinish, final boolean initialize, final long runWait); diff --git a/nifi/nifi-mock/src/test/java/org/apache/nifi/util/CurrentTestStandardProcessorTestRunner.java b/nifi/nifi-mock/src/test/java/org/apache/nifi/util/CurrentTestStandardProcessorTestRunner.java index 450abd6a77..6b403af25c 100644 --- a/nifi/nifi-mock/src/test/java/org/apache/nifi/util/CurrentTestStandardProcessorTestRunner.java +++ b/nifi/nifi-mock/src/test/java/org/apache/nifi/util/CurrentTestStandardProcessorTestRunner.java @@ -1,71 +1,71 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.util; - -import org.apache.nifi.processor.AbstractProcessor; -import org.apache.nifi.processor.ProcessContext; -import org.apache.nifi.processor.ProcessSession; -import org.apache.nifi.processor.exception.ProcessException; -import org.junit.Assert; -import org.junit.Test; - -public class CurrentTestStandardProcessorTestRunner { - - /** - * This test will verify that all iterations of the run are finished before unscheduled is called - */ - @Test - public void testOnScheduledCalledAfterRunFinished() { - SlowRunProcessor processor = new SlowRunProcessor(); - StandardProcessorTestRunner runner = new StandardProcessorTestRunner(processor); - final int iterations = 5; - runner.run(iterations); - // if the counter is not equal to iterations, the the processor must have been unscheduled - // before all the run calls were made, that would be bad. - Assert.assertEquals(iterations, processor.getCounter()); - } - - /** - * This processor simulates a "slow" processor that checks whether it is scheduled before doing something - * - * - */ - private static class SlowRunProcessor extends AbstractProcessor { - - private int counter = 0; - - @Override - public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { - - try { - // be slow - Thread.sleep(50); - // make sure we are still scheduled - if (isScheduled()) { - // increment counter - ++counter; - } - } catch (InterruptedException e) { - } - - } - - public int getCounter() { - return counter; - } - } -} +/* + * 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.util; + +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.exception.ProcessException; +import org.junit.Assert; +import org.junit.Test; + +public class CurrentTestStandardProcessorTestRunner { + + /** + * This test will verify that all iterations of the run are finished before unscheduled is called + */ + @Test + public void testOnScheduledCalledAfterRunFinished() { + SlowRunProcessor processor = new SlowRunProcessor(); + StandardProcessorTestRunner runner = new StandardProcessorTestRunner(processor); + final int iterations = 5; + runner.run(iterations); + // if the counter is not equal to iterations, the the processor must have been unscheduled + // before all the run calls were made, that would be bad. + Assert.assertEquals(iterations, processor.getCounter()); + } + + /** + * This processor simulates a "slow" processor that checks whether it is scheduled before doing something + * + * + */ + private static class SlowRunProcessor extends AbstractProcessor { + + private int counter = 0; + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + + try { + // be slow + Thread.sleep(50); + // make sure we are still scheduled + if (isScheduled()) { + // increment counter + ++counter; + } + } catch (InterruptedException e) { + } + + } + + public int getCounter() { + return counter; + } + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml index a92aebf479..340f2c54b3 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml @@ -35,5 +35,10 @@ nifi-processor-utils test + + commons-io + commons-io + test + diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java index ea033c5ebb..7c32bf8004 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java @@ -70,13 +70,12 @@ public class DocGenerator { for (final Class extensionClass : extensionClasses) { if (ConfigurableComponent.class.isAssignableFrom(extensionClass)) { - final Class componentClass = extensionClass - .asSubclass(ConfigurableComponent.class); + final Class componentClass = extensionClass.asSubclass(ConfigurableComponent.class); try { logger.debug("Documenting: " + componentClass); document(explodedNiFiDocsDir, componentClass); } catch (Exception e) { - logger.warn("Unable to document: " + componentClass); + logger.warn("Unable to document: " + componentClass, e); } } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java index 3c66485923..976c7a5ea4 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java @@ -16,15 +16,21 @@ */ package org.apache.nifi.documentation.init; +import org.apache.nifi.annotation.lifecycle.OnRemoved; +import org.apache.nifi.annotation.lifecycle.OnShutdown; import org.apache.nifi.components.ConfigurableComponent; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.documentation.ConfigurableComponentInitializer; +import org.apache.nifi.documentation.mock.MockConfigurationContext; import org.apache.nifi.documentation.mock.MockControllerServiceInitializationContext; +import org.apache.nifi.documentation.mock.MockProcessorLogger; +import org.apache.nifi.documentation.util.ReflectionUtils; +import org.apache.nifi.logging.ProcessorLog; +import org.apache.nifi.nar.NarCloseable; import org.apache.nifi.reporting.InitializationException; /** - * Initializes a ControllerService using a - * MockControllerServiceInitializationContext + * Initializes a ControllerService using a MockControllerServiceInitializationContext * * */ @@ -33,6 +39,14 @@ public class ControllerServiceInitializer implements ConfigurableComponentInitia @Override public void initialize(ConfigurableComponent component) throws InitializationException { ControllerService controllerService = (ControllerService) component; - controllerService.initialize(new MockControllerServiceInitializationContext()); + + try (NarCloseable narCloseable = NarCloseable.withNarLoader()) { + controllerService.initialize(new MockControllerServiceInitializationContext()); + + final ProcessorLog logger = new MockProcessorLogger(); + final MockConfigurationContext context = new MockConfigurationContext(); + ReflectionUtils.quietlyInvokeMethodsWithAnnotations(OnRemoved.class, null, controllerService, logger, context); + ReflectionUtils.quietlyInvokeMethodsWithAnnotations(OnShutdown.class, null, controllerService, logger, context); + } } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ProcessorInitializer.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ProcessorInitializer.java index 07e9c3ab8f..7f3357c30f 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ProcessorInitializer.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ProcessorInitializer.java @@ -16,9 +16,16 @@ */ package org.apache.nifi.documentation.init; +import org.apache.nifi.annotation.lifecycle.OnRemoved; +import org.apache.nifi.annotation.lifecycle.OnShutdown; import org.apache.nifi.components.ConfigurableComponent; import org.apache.nifi.documentation.ConfigurableComponentInitializer; +import org.apache.nifi.documentation.mock.MockProcessContext; import org.apache.nifi.documentation.mock.MockProcessorInitializationContext; +import org.apache.nifi.documentation.mock.MockProcessorLogger; +import org.apache.nifi.documentation.util.ReflectionUtils; +import org.apache.nifi.logging.ProcessorLog; +import org.apache.nifi.nar.NarCloseable; import org.apache.nifi.processor.Processor; /** @@ -31,7 +38,13 @@ public class ProcessorInitializer implements ConfigurableComponentInitializer { @Override public void initialize(ConfigurableComponent component) { Processor processor = (Processor) component; - processor.initialize(new MockProcessorInitializationContext()); - } + try (NarCloseable narCloseable = NarCloseable.withNarLoader()) { + processor.initialize(new MockProcessorInitializationContext()); + final ProcessorLog logger = new MockProcessorLogger(); + final MockProcessContext context = new MockProcessContext(); + ReflectionUtils.quietlyInvokeMethodsWithAnnotations(OnRemoved.class, null, processor, logger, context); + ReflectionUtils.quietlyInvokeMethodsWithAnnotations(OnShutdown.class, null, processor, logger, context); + } + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ReportingTaskingInitializer.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ReportingTaskingInitializer.java index 6fcfca91da..90642e4828 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ReportingTaskingInitializer.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ReportingTaskingInitializer.java @@ -16,9 +16,15 @@ */ package org.apache.nifi.documentation.init; +import org.apache.nifi.annotation.lifecycle.OnRemoved; +import org.apache.nifi.annotation.lifecycle.OnShutdown; import org.apache.nifi.components.ConfigurableComponent; import org.apache.nifi.documentation.ConfigurableComponentInitializer; +import org.apache.nifi.documentation.mock.MockConfigurationContext; +import org.apache.nifi.documentation.mock.MockProcessorLogger; import org.apache.nifi.documentation.mock.MockReportingInitializationContext; +import org.apache.nifi.documentation.util.ReflectionUtils; +import org.apache.nifi.nar.NarCloseable; import org.apache.nifi.reporting.InitializationException; import org.apache.nifi.reporting.ReportingTask; @@ -32,6 +38,12 @@ public class ReportingTaskingInitializer implements ConfigurableComponentInitial @Override public void initialize(ConfigurableComponent component) throws InitializationException { ReportingTask reportingTask = (ReportingTask) component; - reportingTask.initialize(new MockReportingInitializationContext()); + try (NarCloseable narCloseable = NarCloseable.withNarLoader()) { + reportingTask.initialize(new MockReportingInitializationContext()); + + final MockConfigurationContext context = new MockConfigurationContext(); + ReflectionUtils.quietlyInvokeMethodsWithAnnotations(OnRemoved.class, null, reportingTask, new MockProcessorLogger(), context); + ReflectionUtils.quietlyInvokeMethodsWithAnnotations(OnShutdown.class, null, reportingTask, new MockProcessorLogger(), context); + } } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockConfigurationContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockConfigurationContext.java new file mode 100644 index 0000000000..1ba54999f2 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockConfigurationContext.java @@ -0,0 +1,38 @@ +/* + * 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.documentation.mock; + +import java.util.Collections; +import java.util.Map; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.controller.ConfigurationContext; + +public class MockConfigurationContext implements ConfigurationContext { + + @Override + public PropertyValue getProperty(PropertyDescriptor property) { + return null; + } + + @Override + public Map getProperties() { + return Collections.emptyMap(); + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockProcessContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockProcessContext.java new file mode 100644 index 0000000000..76d7d3d447 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockProcessContext.java @@ -0,0 +1,85 @@ +/* + * 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.documentation.mock; + +import java.util.Collections; +import java.util.Map; +import java.util.Set; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.controller.ControllerServiceLookup; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.Relationship; + +public class MockProcessContext implements ProcessContext { + + @Override + public PropertyValue getProperty(PropertyDescriptor descriptor) { + return null; + } + + @Override + public PropertyValue getProperty(String propertyName) { + return null; + } + + @Override + public PropertyValue newPropertyValue(String rawValue) { + return null; + } + + @Override + public void yield() { + + } + + @Override + public int getMaxConcurrentTasks() { + return 0; + } + + @Override + public String getAnnotationData() { + return ""; + } + + @Override + public Map getProperties() { + return Collections.emptyMap(); + } + + @Override + public String encrypt(String unencrypted) { + return unencrypted; + } + + @Override + public String decrypt(String encrypted) { + return encrypted; + } + + @Override + public ControllerServiceLookup getControllerServiceLookup() { + return new MockControllerServiceLookup(); + } + + @Override + public Set getAvailableRelationships() { + return Collections.emptySet(); + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/util/ReflectionUtils.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/util/ReflectionUtils.java new file mode 100644 index 0000000000..449bd075f7 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/util/ReflectionUtils.java @@ -0,0 +1,139 @@ +/* + * 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.documentation.util; + +import java.lang.annotation.Annotation; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.List; + +import org.apache.nifi.logging.ProcessorLog; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class is a copy of org.apache.nifi.util.ReflectionUtils. Ultimately the documentation generation + * component should be moved to a place where it can depend on this directly instead of copying it in. + * + * + */ +public class ReflectionUtils { + + private final static Logger LOG = LoggerFactory.getLogger(ReflectionUtils.class); + /** + * Invokes all methods on the given instance that have been annotated with the given preferredAnnotation and if no such method exists will invoke all methods on the given instance that have been + * annotated with the given alternateAnnotation, if any exists. If the signature of the method that is defined in instance uses 1 or more parameters, those parameters must be + * specified by the args parameter. However, if more arguments are supplied by the args parameter than needed, the extra arguments will be ignored. + * + * @param preferredAnnotation preferred + * @param alternateAnnotation alternate + * @param instance instance + * @param logger the ProcessorLog to use for logging any errors. If null, will use own logger, but that will not generate bulletins or easily tie to the Processor's log messages. + * @param args args + * @return true if all appropriate methods were invoked and returned without throwing an Exception, false if one of the methods threw an Exception or could not be + * invoked; if false is returned, an error will have been logged. + */ + public static boolean quietlyInvokeMethodsWithAnnotations( + final Class preferredAnnotation, final Class alternateAnnotation, final Object instance, final ProcessorLog logger, final Object... args) { + final List> annotationClasses = new ArrayList<>(alternateAnnotation == null ? 1 : 2); + annotationClasses.add(preferredAnnotation); + if (alternateAnnotation != null) { + annotationClasses.add(alternateAnnotation); + } + + boolean annotationFound = false; + for (final Class annotationClass : annotationClasses) { + if (annotationFound) { + break; + } + + for (final Method method : instance.getClass().getMethods()) { + if (method.isAnnotationPresent(annotationClass)) { + annotationFound = true; + + final boolean isAccessible = method.isAccessible(); + method.setAccessible(true); + + try { + final Class[] argumentTypes = method.getParameterTypes(); + if (argumentTypes.length > args.length) { + if (logger == null) { + LOG.error("Unable to invoke method {} on {} because method expects {} parameters but only {} were given", + new Object[]{method.getName(), instance, argumentTypes.length, args.length}); + } else { + logger.error("Unable to invoke method {} on {} because method expects {} parameters but only {} were given", + new Object[]{method.getName(), instance, argumentTypes.length, args.length}); + } + + return false; + } + + for (int i = 0; i < argumentTypes.length; i++) { + final Class argType = argumentTypes[i]; + if (!argType.isAssignableFrom(args[i].getClass())) { + if (logger == null) { + LOG.error("Unable to invoke method {} on {} because method parameter {} is expected to be of type {} but argument passed was of type {}", + new Object[]{method.getName(), instance, i, argType, args[i].getClass()}); + } else { + logger.error("Unable to invoke method {} on {} because method parameter {} is expected to be of type {} but argument passed was of type {}", + new Object[]{method.getName(), instance, i, argType, args[i].getClass()}); + } + + return false; + } + } + + try { + if (argumentTypes.length == args.length) { + method.invoke(instance, args); + } else { + final Object[] argsToPass = new Object[argumentTypes.length]; + for (int i = 0; i < argsToPass.length; i++) { + argsToPass[i] = args[i]; + } + + method.invoke(instance, argsToPass); + } + } catch (final InvocationTargetException ite) { + if (logger == null) { + LOG.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, ite.getCause()}); + LOG.error("", ite.getCause()); + } else { + logger.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, ite.getCause()}); + } + } catch (final IllegalAccessException | IllegalArgumentException t) { + if (logger == null) { + LOG.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, t}); + LOG.error("", t); + } else { + logger.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, t}); + } + + return false; + } + } finally { + if (!isAccessible) { + method.setAccessible(false); + } + } + } + } + } + return true; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/DocGeneratorTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/DocGeneratorTest.java new file mode 100644 index 0000000000..9438601184 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/DocGeneratorTest.java @@ -0,0 +1,96 @@ +/* + * 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.documentation; + +import java.io.BufferedInputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; + +import org.apache.commons.io.FileUtils; +import org.apache.nifi.nar.ExtensionManager; +import org.apache.nifi.nar.NarClassLoaders; +import org.apache.nifi.nar.NarUnpacker; +import org.apache.nifi.util.NiFiProperties; +import org.junit.Assert; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class DocGeneratorTest { + + @Test + public void testProcessorLoadsNarResources() throws IOException, ClassNotFoundException { + TemporaryFolder temporaryFolder = new TemporaryFolder(); + temporaryFolder.create(); + + NiFiProperties properties = loadSpecifiedProperties("/conf/nifi.properties"); + properties.setProperty(NiFiProperties.COMPONENT_DOCS_DIRECTORY, temporaryFolder.getRoot().getAbsolutePath()); + + NarUnpacker.unpackNars(properties); + + NarClassLoaders.load(properties); + + ExtensionManager.discoverExtensions(); + + DocGenerator.generate(properties); + + File processorDirectory = new File(temporaryFolder.getRoot(), "org.apache.nifi.processors.WriteResourceToStream"); + File indexHtml = new File(processorDirectory, "index.html"); + Assert.assertTrue(indexHtml + " should have been generated", indexHtml.exists()); + String generatedHtml = FileUtils.readFileToString(indexHtml); + Assert.assertNotNull(generatedHtml); + Assert.assertTrue(generatedHtml.contains("This example processor loads a resource from the nar and writes it to the FlowFile content")); + Assert.assertTrue(generatedHtml.contains("files that were successfully processed")); + Assert.assertTrue(generatedHtml.contains("files that were not successfully processed")); + Assert.assertTrue(generatedHtml.contains("resources")); + } + + private NiFiProperties loadSpecifiedProperties(String propertiesFile) { + String file = DocGeneratorTest.class.getResource(propertiesFile).getFile(); + + System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, file); + + NiFiProperties properties = NiFiProperties.getInstance(); + + // clear out existing properties + for (String prop : properties.stringPropertyNames()) { + properties.remove(prop); + } + + InputStream inStream = null; + try { + inStream = new BufferedInputStream(new FileInputStream(file)); + properties.load(inStream); + } catch (final Exception ex) { + throw new RuntimeException("Cannot load properties file due to " + + ex.getLocalizedMessage(), ex); + } finally { + if (null != inStream) { + try { + inStream.close(); + } catch (final Exception ex) { + /** + * do nothing * + */ + } + } + } + + return properties; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedControllerService.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedControllerService.java index cd68267d30..ea5f622b04 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedControllerService.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedControllerService.java @@ -22,26 +22,32 @@ import java.util.List; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnRemoved; +import org.apache.nifi.annotation.lifecycle.OnShutdown; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.processor.util.StandardValidators; -@CapabilityDescription("A documented controller service that can help you do things") -@Tags({"one", "two", "three"}) -public class FullyDocumentedControllerService extends AbstractControllerService implements SampleService{ +@CapabilityDescription("A documented controller service that can help you do things") +@Tags({ "one", "two", "three" }) +public class FullyDocumentedControllerService extends AbstractControllerService implements SampleService { - public static final PropertyDescriptor KEYSTORE = new PropertyDescriptor.Builder().name("Keystore Filename") - .description("The fully-qualified filename of the Keystore").defaultValue(null) + public static final PropertyDescriptor KEYSTORE = new PropertyDescriptor.Builder().name("Keystore Filename").description("The fully-qualified filename of the Keystore").defaultValue(null) .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR).sensitive(false).build(); - public static final PropertyDescriptor KEYSTORE_TYPE = new PropertyDescriptor.Builder().name("Keystore Type") - .description("The Type of the Keystore").allowableValues("JKS", "PKCS12") + public static final PropertyDescriptor KEYSTORE_TYPE = new PropertyDescriptor.Builder().name("Keystore Type").description("The Type of the Keystore").allowableValues("JKS", "PKCS12") .addValidator(StandardValidators.NON_EMPTY_VALIDATOR).defaultValue("JKS").sensitive(false).build(); - public static final PropertyDescriptor KEYSTORE_PASSWORD = new PropertyDescriptor.Builder() - .name("Keystore Password").defaultValue(null).description("The password for the Keystore") + public static final PropertyDescriptor KEYSTORE_PASSWORD = new PropertyDescriptor.Builder().name("Keystore Password").defaultValue(null).description("The password for the Keystore") .addValidator(StandardValidators.NON_EMPTY_VALIDATOR).sensitive(true).build(); private static final List properties; + private int onRemovedNoArgs = 0; + private int onRemovedArgs = 0; + + private int onShutdownNoArgs = 0; + private int onShutdownArgs = 0; + static { List props = new ArrayList<>(); props.add(KEYSTORE); @@ -53,10 +59,45 @@ public class FullyDocumentedControllerService extends AbstractControllerService @Override protected List getSupportedPropertyDescriptors() { return properties; - } + } @Override public void doSomething() { - // TODO Auto-generated method stub + } + + @OnRemoved + public void onRemovedNoArgs() { + onRemovedNoArgs++; + } + + @OnRemoved + public void onRemovedArgs(ConfigurationContext context) { + onRemovedArgs++; + } + + @OnShutdown + public void onShutdownNoArgs() { + onShutdownNoArgs++; + } + + @OnShutdown + public void onShutdownArgs(ConfigurationContext context) { + onShutdownArgs++; + } + + public int getOnRemovedNoArgs() { + return onRemovedNoArgs; + } + + public int getOnRemovedArgs() { + return onRemovedArgs; + } + + public int getOnShutdownNoArgs() { + return onShutdownNoArgs; + } + + public int getOnShutdownArgs() { + return onShutdownArgs; } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java index 178a144534..379c10bad1 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java @@ -23,13 +23,15 @@ import java.util.List; import java.util.Set; import org.apache.nifi.annotation.behavior.DynamicProperty; -import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.behavior.DynamicRelationship; import org.apache.nifi.annotation.behavior.ReadsAttribute; -import org.apache.nifi.annotation.documentation.SeeAlso; -import org.apache.nifi.annotation.documentation.Tags; 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.SeeAlso; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnRemoved; +import org.apache.nifi.annotation.lifecycle.OnShutdown; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.processor.AbstractProcessor; @@ -88,6 +90,12 @@ public class FullyDocumentedProcessor extends AbstractProcessor { private List properties; private Set relationships; + private int onRemovedNoArgs = 0; + private int onRemovedArgs = 0; + + private int onShutdownNoArgs = 0; + private int onShutdownArgs = 0; + @Override protected void init(ProcessorInitializationContext context) { final List properties = new ArrayList<>(); @@ -126,4 +134,39 @@ public class FullyDocumentedProcessor extends AbstractProcessor { .description("This is a property you can use or not").dynamic(true).build(); } + @OnRemoved + public void onRemovedNoArgs() { + onRemovedNoArgs++; + } + + @OnRemoved + public void onRemovedArgs(ProcessContext context) { + onRemovedArgs++; + } + + @OnShutdown + public void onShutdownNoArgs() { + onShutdownNoArgs++; + } + + @OnShutdown + public void onShutdownArgs(ProcessContext context) { + onShutdownArgs++; + } + + public int getOnRemovedNoArgs() { + return onRemovedNoArgs; + } + + public int getOnRemovedArgs() { + return onRemovedArgs; + } + + public int getOnShutdownNoArgs() { + return onShutdownNoArgs; + } + + public int getOnShutdownArgs() { + return onShutdownArgs; + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedReportingTask.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedReportingTask.java index 17e446af6c..43929be9c2 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedReportingTask.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedReportingTask.java @@ -20,8 +20,11 @@ import java.util.ArrayList; import java.util.List; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnRemoved; +import org.apache.nifi.annotation.lifecycle.OnShutdown; import org.apache.nifi.reporting.AbstractReportingTask; import org.apache.nifi.reporting.ReportingContext; @@ -35,6 +38,12 @@ public class FullyDocumentedReportingTask extends AbstractReportingTask { "Specifies whether or not to show the difference in values between the current status and the previous status") .required(true).allowableValues("true", "false").defaultValue("true").build(); + private int onRemovedNoArgs = 0; + private int onRemovedArgs = 0; + + private int onShutdownNoArgs = 0; + private int onShutdownArgs = 0; + @Override public final List getSupportedPropertyDescriptors() { final List descriptors = new ArrayList<>(); @@ -44,7 +53,42 @@ public class FullyDocumentedReportingTask extends AbstractReportingTask { @Override public void onTrigger(ReportingContext context) { - // TODO Auto-generated method stub } + + @OnRemoved + public void onRemovedNoArgs() { + onRemovedNoArgs++; + } + + @OnRemoved + public void onRemovedArgs(ConfigurationContext context) { + onRemovedArgs++; + } + + @OnShutdown + public void onShutdownNoArgs() { + onShutdownNoArgs++; + } + + @OnShutdown + public void onShutdownArgs(ConfigurationContext context) { + onShutdownArgs++; + } + + public int getOnRemovedNoArgs() { + return onRemovedNoArgs; + } + + public int getOnRemovedArgs() { + return onRemovedArgs; + } + + public int getOnShutdownNoArgs() { + return onShutdownNoArgs; + } + + public int getOnShutdownArgs() { + return onShutdownArgs; + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java index 9179ed2323..d85e19b7e5 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java @@ -28,26 +28,30 @@ import org.apache.nifi.documentation.example.ControllerServiceWithLogger; import org.apache.nifi.documentation.example.FullyDocumentedControllerService; import org.apache.nifi.documentation.example.FullyDocumentedReportingTask; import org.apache.nifi.documentation.example.ReportingTaskWithLogger; +import org.apache.nifi.documentation.init.ControllerServiceInitializer; +import org.apache.nifi.documentation.init.ReportingTaskingInitializer; import org.apache.nifi.documentation.mock.MockControllerServiceInitializationContext; import org.apache.nifi.documentation.mock.MockReportingInitializationContext; import org.apache.nifi.reporting.InitializationException; import org.apache.nifi.reporting.ReportingTask; +import org.junit.Assert; import org.junit.Test; public class HtmlDocumentationWriterTest { @Test public void testJoin() { - assertEquals("a, b, c", HtmlDocumentationWriter.join(new String[]{"a", "b", "c"}, ", ")); - assertEquals("a, b", HtmlDocumentationWriter.join(new String[]{"a", "b"}, ", ")); - assertEquals("a", HtmlDocumentationWriter.join(new String[]{"a"}, ", ")); + assertEquals("a, b, c", HtmlDocumentationWriter.join(new String[] { "a", "b", "c" }, ", ")); + assertEquals("a, b", HtmlDocumentationWriter.join(new String[] { "a", "b" }, ", ")); + assertEquals("a", HtmlDocumentationWriter.join(new String[] { "a" }, ", ")); } @Test public void testDocumentControllerService() throws InitializationException, IOException { - ControllerService controllerService = new FullyDocumentedControllerService(); - controllerService.initialize(new MockControllerServiceInitializationContext()); + FullyDocumentedControllerService controllerService = new FullyDocumentedControllerService(); + ControllerServiceInitializer initializer = new ControllerServiceInitializer(); + initializer.initialize(controllerService); DocumentationWriter writer = new HtmlDocumentationWriter(); @@ -71,13 +75,21 @@ public class HtmlDocumentationWriterTest { assertContains(results, "JKS"); assertContains(results, "PKCS12"); assertContains(results, "Sensitive Property: true"); + + // verify the right OnRemoved and OnShutdown methods were called + Assert.assertEquals(1, controllerService.getOnRemovedArgs()); + Assert.assertEquals(1, controllerService.getOnRemovedNoArgs()); + + Assert.assertEquals(1, controllerService.getOnShutdownArgs()); + Assert.assertEquals(1, controllerService.getOnShutdownNoArgs()); } @Test public void testDocumentReportingTask() throws InitializationException, IOException { - ReportingTask reportingTask = new FullyDocumentedReportingTask(); - reportingTask.initialize(new MockReportingInitializationContext()); + FullyDocumentedReportingTask reportingTask = new FullyDocumentedReportingTask(); + ReportingTaskingInitializer initializer = new ReportingTaskingInitializer(); + initializer.initialize(reportingTask); DocumentationWriter writer = new HtmlDocumentationWriter(); @@ -99,6 +111,13 @@ public class HtmlDocumentationWriterTest { assertContains(results, "Specifies whether or not to show the difference in values between the current status and the previous status"); assertContains(results, "true"); assertContains(results, "false"); + + // verify the right OnRemoved and OnShutdown methods were called + Assert.assertEquals(1, reportingTask.getOnRemovedArgs()); + Assert.assertEquals(1, reportingTask.getOnRemovedNoArgs()); + + Assert.assertEquals(1, reportingTask.getOnShutdownArgs()); + Assert.assertEquals(1, reportingTask.getOnShutdownNoArgs()); } @Test diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java index c421148612..fda81fd4e7 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java @@ -27,7 +27,8 @@ import org.apache.nifi.documentation.DocumentationWriter; import org.apache.nifi.documentation.example.FullyDocumentedProcessor; import org.apache.nifi.documentation.example.NakedProcessor; import org.apache.nifi.documentation.example.ProcessorWithLogger; -import org.apache.nifi.documentation.mock.MockProcessorInitializationContext; +import org.apache.nifi.documentation.init.ProcessorInitializer; +import org.junit.Assert; import org.junit.Test; public class ProcessorDocumentationWriterTest { @@ -35,7 +36,8 @@ public class ProcessorDocumentationWriterTest { @Test public void testFullyDocumentedProcessor() throws IOException { FullyDocumentedProcessor processor = new FullyDocumentedProcessor(); - processor.initialize(new MockProcessorInitializationContext()); + ProcessorInitializer initializer = new ProcessorInitializer(); + initializer.initialize(processor); DocumentationWriter writer = new HtmlProcessorDocumentationWriter(); @@ -70,12 +72,20 @@ public class ProcessorDocumentationWriterTest { assertNotContains(results, "No description provided."); assertNotContains(results, "No Tags provided."); assertNotContains(results, "Additional Details..."); + + // verify the right OnRemoved and OnShutdown methods were called + Assert.assertEquals(1, processor.getOnRemovedArgs()); + Assert.assertEquals(1, processor.getOnRemovedNoArgs()); + + Assert.assertEquals(1, processor.getOnShutdownArgs()); + Assert.assertEquals(1, processor.getOnShutdownNoArgs()); } @Test public void testNakedProcessor() throws IOException { NakedProcessor processor = new NakedProcessor(); - processor.initialize(new MockProcessorInitializationContext()); + ProcessorInitializer initializer = new ProcessorInitializer(); + initializer.initialize(processor); DocumentationWriter writer = new HtmlProcessorDocumentationWriter(); @@ -103,7 +113,8 @@ public class ProcessorDocumentationWriterTest { @Test public void testProcessorWithLoggerInitialization() throws IOException { ProcessorWithLogger processor = new ProcessorWithLogger(); - processor.initialize(new MockProcessorInitializationContext()); + ProcessorInitializer initializer = new ProcessorInitializer(); + initializer.initialize(processor); DocumentationWriter writer = new HtmlProcessorDocumentationWriter(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/resources/conf/nifi.properties b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/resources/conf/nifi.properties new file mode 100644 index 0000000000..a0e13895eb --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/resources/conf/nifi.properties @@ -0,0 +1,129 @@ +# 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. + +# Core Properties # +nifi.version=nifi-test 3.0.0 +nifi.flow.configuration.file=./target/flow.xml.gz +nifi.flow.configuration.archive.dir=./target/archive/ +nifi.flowcontroller.autoResumeState=true +nifi.flowcontroller.graceful.shutdown.period=10 sec +nifi.flowservice.writedelay.interval=2 sec +nifi.administrative.yield.duration=30 sec + +nifi.reporting.task.configuration.file=./target/reporting-tasks.xml +nifi.controller.service.configuration.file=./target/controller-services.xml +nifi.templates.directory=./target/templates +nifi.ui.banner.text=UI Banner Text +nifi.ui.autorefresh.interval=30 sec +nifi.nar.library.directory=./target/test-classes/lib/ +nifi.nar.library.directory.alt=./target/test-classes/lib2/ + +nifi.nar.working.directory=./target/work/nar/ + +# H2 Settings +nifi.database.directory=./target/database_repository +nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE + +# FlowFile Repository +nifi.flowfile.repository.directory=./target/test-repo +nifi.flowfile.repository.partitions=1 +nifi.flowfile.repository.checkpoint.interval=2 mins +nifi.queue.swap.threshold=20000 +nifi.swap.storage.directory=./target/test-repo/swap +nifi.swap.in.period=5 sec +nifi.swap.in.threads=1 +nifi.swap.out.period=5 sec +nifi.swap.out.threads=4 + +# Content Repository +nifi.content.claim.max.appendable.size=10 MB +nifi.content.claim.max.flow.files=100 +nifi.content.repository.directory.default=./target/content_repository + +# Provenance Repository Properties +nifi.provenance.repository.storage.directory=./target/provenance_repository +nifi.provenance.repository.max.storage.time=24 hours +nifi.provenance.repository.max.storage.size=1 GB +nifi.provenance.repository.rollover.time=5 mins +nifi.provenance.repository.rollover.size=100 MB + +# Site to Site properties +nifi.remote.input.socket.port=9990 +nifi.remote.input.secure=true + +# web properties # +nifi.web.war.directory=./target/lib +nifi.web.http.host= +nifi.web.http.port=8080 +nifi.web.https.host= +nifi.web.https.port= +nifi.web.jetty.working.directory=./target/work/jetty + +# security properties # +nifi.sensitive.props.key=key +nifi.sensitive.props.algorithm=PBEWITHMD5AND256BITAES-CBC-OPENSSL +nifi.sensitive.props.provider=BC + +nifi.security.keystore= +nifi.security.keystoreType= +nifi.security.keystorePasswd= +nifi.security.keyPasswd= +nifi.security.truststore= +nifi.security.truststoreType= +nifi.security.truststorePasswd= +nifi.security.needClientAuth= +nifi.security.authorizedUsers.file=./target/conf/authorized-users.xml +nifi.security.user.credential.cache.duration=24 hours +nifi.security.user.authority.provider=nifi.authorization.FileAuthorizationProvider +nifi.security.support.new.account.requests= +nifi.security.default.user.roles= + +# cluster common properties (cluster manager and nodes must have same values) # +nifi.cluster.protocol.heartbeat.interval=5 sec +nifi.cluster.protocol.is.secure=false +nifi.cluster.protocol.socket.timeout=30 sec +nifi.cluster.protocol.connection.handshake.timeout=45 sec +# if multicast is used, then nifi.cluster.protocol.multicast.xxx properties must be configured # +nifi.cluster.protocol.use.multicast=false +nifi.cluster.protocol.multicast.address= +nifi.cluster.protocol.multicast.port= +nifi.cluster.protocol.multicast.service.broadcast.delay=500 ms +nifi.cluster.protocol.multicast.service.locator.attempts=3 +nifi.cluster.protocol.multicast.service.locator.attempts.delay=1 sec + +# cluster node properties (only configure for cluster nodes) # +nifi.cluster.is.node=false +nifi.cluster.node.address= +nifi.cluster.node.protocol.port= +nifi.cluster.node.protocol.threads=2 +# if multicast is not used, nifi.cluster.node.unicast.xxx must have same values as nifi.cluster.manager.xxx # +nifi.cluster.node.unicast.manager.address= +nifi.cluster.node.unicast.manager.protocol.port= +nifi.cluster.node.unicast.manager.authority.provider.port= + +# cluster manager properties (only configure for cluster manager) # +nifi.cluster.is.manager=false +nifi.cluster.manager.address= +nifi.cluster.manager.protocol.port= +nifi.cluster.manager.authority.provider.port= +nifi.cluster.manager.authority.provider.threads=10 +nifi.cluster.manager.node.firewall.file= +nifi.cluster.manager.node.event.history.size=10 +nifi.cluster.manager.node.api.connection.timeout=30 sec +nifi.cluster.manager.node.api.read.timeout=30 sec +nifi.cluster.manager.node.api.request.threads=10 +nifi.cluster.manager.flow.retrieval.delay=5 sec +nifi.cluster.manager.protocol.threads=10 +nifi.cluster.manager.safemode.duration=0 sec diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/resources/lib/example.nar b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/resources/lib/example.nar new file mode 100644 index 0000000000..0bca10b513 Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/resources/lib/example.nar differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/resources/lib/jetty.nar b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/resources/lib/jetty.nar new file mode 100644 index 0000000000..f92d6154d1 Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/resources/lib/jetty.nar differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/resources/lib/nifi-framework-nar.nar b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/resources/lib/nifi-framework-nar.nar new file mode 100644 index 0000000000..d2a8b9667e Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/resources/lib/nifi-framework-nar.nar differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java index c0395a42a1..335c0ef5a4 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java @@ -188,7 +188,7 @@ public class DataFlowDaoImpl implements DataFlowDao { return; } - if ((primaryEntry == null && restoreEntry != null) || (primaryEntry != null && restoreEntry == null)) { + if (primaryEntry == null && restoreEntry != null || primaryEntry != null && restoreEntry == null) { throw new IllegalStateException(String.format("Primary file '%s' is different than restore file '%s'", primaryFile.getAbsoluteFile(), restoreFile.getAbsolutePath())); } @@ -352,7 +352,7 @@ public class DataFlowDaoImpl implements DataFlowDao { final File[] files = dir.listFiles(new FilenameFilter() { @Override public boolean accept(File dir, String name) { - return (name.equals(FLOW_PACKAGE) || name.endsWith(STALE_EXT) || name.endsWith(UNKNOWN_EXT)); + return name.equals(FLOW_PACKAGE) || name.endsWith(STALE_EXT) || name.endsWith(UNKNOWN_EXT); } }); @@ -515,19 +515,10 @@ public class DataFlowDaoImpl implements DataFlowDao { final StandardDataFlow dataFlow = new StandardDataFlow(flowBytes, templateBytes, snippetBytes); dataFlow.setAutoStartProcessors(autoStart); - return new ClusterDataFlow(dataFlow, (clusterMetadata == null) ? null : clusterMetadata.getPrimaryNodeId(), controllerServiceBytes, reportingTaskBytes); + return new ClusterDataFlow(dataFlow, clusterMetadata == null ? null : clusterMetadata.getPrimaryNodeId(), controllerServiceBytes, reportingTaskBytes); } private void writeDataFlow(final File file, final ClusterDataFlow clusterDataFlow) throws IOException, JAXBException { - - // get the data flow - DataFlow dataFlow = clusterDataFlow.getDataFlow(); - - // if no dataflow, then write a new dataflow - if (dataFlow == null) { - dataFlow = new StandardDataFlow(new byte[0], new byte[0], new byte[0]); - } - // setup the cluster metadata final ClusterMetadata clusterMetadata = new ClusterMetadata(); clusterMetadata.setPrimaryNodeId(clusterDataFlow.getPrimaryNodeId()); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java index e212bdbf38..1171636d96 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java @@ -130,7 +130,6 @@ public class FileSystemRepository implements ContentRepository { final String maxArchiveRetentionPeriod = properties.getProperty(NiFiProperties.CONTENT_ARCHIVE_MAX_RETENTION_PERIOD); final String maxArchiveSize = properties.getProperty(NiFiProperties.CONTENT_ARCHIVE_MAX_USAGE_PERCENTAGE); final String archiveBackPressureSize = properties.getProperty(NiFiProperties.CONTENT_ARCHIVE_BACK_PRESSURE_PERCENTAGE); - final String archiveCleanupFrequency = properties.getProperty(NiFiProperties.CONTENT_ARCHIVE_CLEANUP_FREQUENCY); if ("true".equalsIgnoreCase(enableArchiving)) { archiveData = true; @@ -193,11 +192,23 @@ public class FileSystemRepository implements ContentRepository { LOG.info("Initializing FileSystemRepository with 'Always Sync' set to {}", alwaysSync); initializeRepository(); + containerCleanupExecutor = new FlowEngine(containers.size(), "Cleanup FileSystemRepository Container", true); + } + + @Override + public void initialize(final ContentClaimManager claimManager) { + this.contentClaimManager = claimManager; + + final NiFiProperties properties = NiFiProperties.getInstance(); + + final Map fileRespositoryPaths = properties.getContentRepositoryPaths(); + executor.scheduleWithFixedDelay(new BinDestructableClaims(), 1, 1, TimeUnit.SECONDS); for (int i = 0; i < fileRespositoryPaths.size(); i++) { executor.scheduleWithFixedDelay(new ArchiveOrDestroyDestructableClaims(), 1, 1, TimeUnit.SECONDS); } + final String archiveCleanupFrequency = properties.getProperty(NiFiProperties.CONTENT_ARCHIVE_CLEANUP_FREQUENCY); final long cleanupMillis; if (archiveCleanupFrequency == null) { cleanupMillis = 1000L; @@ -208,8 +219,6 @@ public class FileSystemRepository implements ContentRepository { throw new RuntimeException("Invalid value set for property " + NiFiProperties.CONTENT_ARCHIVE_CLEANUP_FREQUENCY); } } - - containerCleanupExecutor = new FlowEngine(containers.size(), "Cleanup FileSystemRepository Container", true); for (final Map.Entry containerEntry : containers.entrySet()) { final String containerName = containerEntry.getKey(); final Path containerPath = containerEntry.getValue(); @@ -218,11 +227,6 @@ public class FileSystemRepository implements ContentRepository { } } - @Override - public void initialize(final ContentClaimManager claimManager) { - this.contentClaimManager = claimManager; - } - @Override public void shutdown() { executor.shutdown(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java index 4ee8c06e2c..04e819e5a2 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java @@ -260,7 +260,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE final FlowFileRecord clone = builder.build(); final StandardRepositoryRecord newRecord = new StandardRepositoryRecord(destination.getFlowFileQueue()); - getProvenanceReporter().clone(currRec, clone); + provenanceReporter.clone(currRec, clone, false); final ContentClaim claim = clone.getContentClaim(); if (claim != null) { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProvenanceReporter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProvenanceReporter.java index 5194fef3b8..8852f424f4 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProvenanceReporter.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProvenanceReporter.java @@ -327,7 +327,13 @@ public class StandardProvenanceReporter implements ProvenanceReporter { @Override public void clone(final FlowFile parent, final FlowFile child) { - verifyFlowFileKnown(child); + clone(parent, child, true); + } + + void clone(final FlowFile parent, final FlowFile child, final boolean verifyFlowFile) { + if (verifyFlowFile) { + verifyFlowFileKnown(child); + } try { final ProvenanceEventBuilder eventBuilder = build(parent, ProvenanceEventType.CLONE); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java index ae00326b6b..3bfdd8a52c 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java @@ -116,15 +116,15 @@ public class VolatileContentRepository implements ContentRepository { } memoryManager = new MemoryManager(maxBytes, blockSize); - - for (int i = 0; i < 3; i++) { - executor.scheduleWithFixedDelay(new CleanupOldClaims(), 1000, 10, TimeUnit.MILLISECONDS); - } } @Override public void initialize(final ContentClaimManager claimManager) { this.claimManager = claimManager; + + for (int i = 0; i < 3; i++) { + executor.scheduleWithFixedDelay(new CleanupOldClaims(), 1000, 10, TimeUnit.MILLISECONDS); + } } @Override diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/META-INF/services/org.apache.nifi.controller.repository.ContentRepository b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/META-INF/services/org.apache.nifi.controller.repository.ContentRepository new file mode 100644 index 0000000000..8f6892010b --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/META-INF/services/org.apache.nifi.controller.repository.ContentRepository @@ -0,0 +1,16 @@ +# 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. +org.apache.nifi.controller.repository.FileSystemRepository +org.apache.nifi.controller.repository.VolatileContentRepository \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/META-INF/services/org.apache.nifi.controller.repository.FlowFileRepository b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/META-INF/services/org.apache.nifi.controller.repository.FlowFileRepository new file mode 100644 index 0000000000..590dbc1871 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/META-INF/services/org.apache.nifi.controller.repository.FlowFileRepository @@ -0,0 +1,16 @@ +# 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. +org.apache.nifi.controller.repository.WriteAheadFlowFileRepository +org.apache.nifi.controller.repository.VolatileFlowFileRepository \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/META-INF/services/org.apache.nifi.controller.repository.FlowFileSwapManager b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/META-INF/services/org.apache.nifi.controller.repository.FlowFileSwapManager new file mode 100644 index 0000000000..e5c63ac8f8 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/META-INF/services/org.apache.nifi.controller.repository.FlowFileSwapManager @@ -0,0 +1,15 @@ +# 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. +org.apache.nifi.controller.FileSystemSwapManager \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/META-INF/services/org.apache.nifi.controller.status.history.ComponentStatusRepository b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/META-INF/services/org.apache.nifi.controller.status.history.ComponentStatusRepository new file mode 100644 index 0000000000..b197ace353 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/META-INF/services/org.apache.nifi.controller.status.history.ComponentStatusRepository @@ -0,0 +1,15 @@ +# 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. +org.apache.nifi.controller.status.history.VolatileComponentStatusRepository \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/nar/NarCloseable.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/nar/NarCloseable.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/bin/nifi.sh b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/bin/nifi.sh index 6d3191d9a7..8caf55e84d 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/bin/nifi.sh +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/bin/nifi.sh @@ -140,6 +140,10 @@ install() { cp $0 $SVC_FILE sed -i s:NIFI_HOME=.*:NIFI_HOME="$NIFI_HOME": $SVC_FILE sed -i s:PROGNAME=.*:PROGNAME=$(basename "$0"): $SVC_FILE + rm -f /etc/rc2.d/S65${SVC_NAME} + ln -s /etc/init.d/$SVC_NAME /etc/rc2.d/S65${SVC_NAME} + rm -f /etc/rc2.d/K65${SVC_NAME} + ln -s /etc/init.d/$SVC_NAME /etc/rc2.d/K65${SVC_NAME} echo Service $SVC_NAME installed } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties index 90b3cdd07f..4043076774 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties @@ -82,6 +82,9 @@ nifi.provenance.repository.indexed.attributes=${nifi.provenance.repository.index # Large values for the shard size will result in more Java heap usage when searching the Provenance Repository # but should provide better performance nifi.provenance.repository.index.shard.size=${nifi.provenance.repository.index.shard.size} +# Indicates the maximum length that a FlowFile attribute can be when retrieving a Provenance Event from +# the repository. If the length of any attribute exceeds this value, it will be truncated when the event is retrieved. +nifi.provenance.repository.max.attribute.length=${nifi.provenance.repository.max.attribute.length} # Volatile Provenance Respository Properties nifi.provenance.repository.buffer.size=${nifi.provenance.repository.buffer.size} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/main.css b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/main.css index e161efc14c..70916c6c8f 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/main.css +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/main.css @@ -115,6 +115,7 @@ div.nifi-tooltip { border: 1px solid #454545; background-color: #FFFFA3; color: #454545; + max-width: 500px; } .ellipsis { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js index ca45a3dbe9..59f38a5ec3 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js @@ -811,7 +811,7 @@ nf.Canvas = (function () { // bulletins for this processor are now gone if (bulletins.length === 0) { if (bulletinIcon.data('qtip')) { - bulletinIcon.removeClass('has-bulletins').qtip('destroy'); + bulletinIcon.removeClass('has-bulletins').qtip('api').destroy(true); } // hide the icon diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-controller-service.js b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-controller-service.js index e8a111d419..65c6dfa1fd 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-controller-service.js +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-controller-service.js @@ -264,7 +264,7 @@ nf.ControllerService = (function () { }, nf.CanvasUtils.config.systemTooltipConfig)); } } else if (icon.data('qtip')) { - icon.qtip('destroy'); + icon.qtip('api').destroy(true); } return state; }); @@ -294,7 +294,7 @@ nf.ControllerService = (function () { }, nf.CanvasUtils.config.systemTooltipConfig)); } } else if (icon.data('qtip')) { - icon.qtip('destroy'); + icon.qtip('api').destroy(true); } return state; }); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js index 2a5273ddfc..110d383057 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js @@ -184,6 +184,23 @@ nf.Common = { * @argument {string} error The error */ handleAjaxError: function (xhr, status, error) { + // if an error occurs while the splash screen is visible close the canvas show the error message + if ($('#splash').is(':visible')) { + $('#message-title').text('An unexpected error has occurred'); + if ($.trim(xhr.responseText) === '') { + $('#message-content').text('Please check the logs.'); + } else { + $('#message-content').text(xhr.responseText); + } + + // show the error pane + $('#message-pane').show(); + + // close the canvas + nf.Common.closeCanvas(); + return; + } + // status code 400, 404, and 409 are expected response codes for common errors. if (xhr.status === 400 || xhr.status === 404 || xhr.status === 409) { nf.Dialog.showOkDialog({ diff --git a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java index 44b65849af..d83c7bfdbb 100644 --- a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java +++ b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.io.InputStream; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Collection; import java.util.HashSet; import java.util.List; import java.util.Properties; @@ -39,6 +40,8 @@ import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.DataUnit; @@ -58,7 +61,7 @@ import org.apache.nifi.util.LongHolder; import scala.actors.threadpool.Arrays; @SupportsBatching -@Tags({"Apache", "Kafka", "Put", "Send", "Message", "PubSub"}) +@Tags({ "Apache", "Kafka", "Put", "Send", "Message", "PubSub" }) @CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Kafka") public class PutKafka extends AbstractProcessor { @@ -66,94 +69,190 @@ public class PutKafka extends AbstractProcessor { private static final String BROKER_REGEX = SINGLE_BROKER_REGEX + "(?:,\\s*" + SINGLE_BROKER_REGEX + ")*"; public static final AllowableValue DELIVERY_REPLICATED = new AllowableValue("-1", "Guarantee Replicated Delivery", "FlowFile will be routed to" - + " failure unless the message is replicated to the appropriate number of Kafka Nodes according to the Topic configuration"); + + " failure unless the message is replicated to the appropriate number of Kafka Nodes according to the Topic configuration"); public static final AllowableValue DELIVERY_ONE_NODE = new AllowableValue("1", "Guarantee Single Node Delivery", "FlowFile will be routed" - + " to success if the message is received by a single Kafka node, whether or not it is replicated. This is faster than" - + " but can result in data loss if a Kafka node crashes"); + + " to success if the message is received by a single Kafka node, whether or not it is replicated. This is faster than" + + " but can result in data loss if a Kafka node crashes"); public static final AllowableValue DELIVERY_BEST_EFFORT = new AllowableValue("0", "Best Effort", "FlowFile will be routed to success after" - + " successfully writing the content to a Kafka node, without waiting for a response. This provides the best performance but may result" - + " in data loss."); + + " successfully writing the content to a Kafka node, without waiting for a response. This provides the best performance but may result" + + " in data loss."); + + /** + * AllowableValue for a Producer Type that synchronously sends messages to Kafka + */ + public static final AllowableValue PRODUCTER_TYPE_SYNCHRONOUS = new AllowableValue("sync", "Synchronous", "Send FlowFiles to Kafka immediately."); + + /** + * AllowableValue for a Producer Type that asynchronously sends messages to Kafka + */ + public static final AllowableValue PRODUCTER_TYPE_ASYNCHRONOUS = new AllowableValue("async", "Asynchronous", "Batch messages before sending them to Kafka." + + " While this will improve throughput, it opens the possibility that a failure on the client machine will drop unsent data."); + + /** + * AllowableValue for sending messages to Kafka without compression + */ + public static final AllowableValue COMPRESSION_CODEC_NONE = new AllowableValue("none", "None", "Compression will not be used for any topic."); + + /** + * AllowableValue for sending messages to Kafka with GZIP compression + */ + public static final AllowableValue COMPRESSION_CODEC_GZIP = new AllowableValue("gzip", "GZIP", "Compress messages using GZIP"); + + /** + * AllowableValue for sending messages to Kafka with Snappy compression + */ + public static final AllowableValue COMPRESSION_CODEC_SNAPPY = new AllowableValue("snappy", "Snappy", "Compress messages using Snappy"); + public static final PropertyDescriptor SEED_BROKERS = new PropertyDescriptor.Builder() - .name("Known Brokers") - .description("A comma-separated list of known Kafka Brokers in the format :") - .required(true) - .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile(BROKER_REGEX))) - .expressionLanguageSupported(false) - .build(); + .name("Known Brokers") + .description("A comma-separated list of known Kafka Brokers in the format :") + .required(true) + .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile(BROKER_REGEX))) + .expressionLanguageSupported(false) + .build(); public static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder() - .name("Topic Name") - .description("The Kafka Topic of interest") - .required(true) - .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) - .expressionLanguageSupported(true) - .build(); + .name("Topic Name") + .description("The Kafka Topic of interest") + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(true) + .build(); public static final PropertyDescriptor KEY = new PropertyDescriptor.Builder() - .name("Kafka Key") - .description("The Key to use for the Message") - .required(false) - .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) - .expressionLanguageSupported(true) - .build(); + .name("Kafka Key") + .description("The Key to use for the Message") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(true) + .build(); public static final PropertyDescriptor DELIVERY_GUARANTEE = new PropertyDescriptor.Builder() - .name("Delivery Guarantee") - .description("Specifies the requirement for guaranteeing that a message is sent to Kafka") - .required(true) - .expressionLanguageSupported(false) - .allowableValues(DELIVERY_BEST_EFFORT, DELIVERY_ONE_NODE, DELIVERY_REPLICATED) - .defaultValue(DELIVERY_BEST_EFFORT.getValue()) - .build(); + .name("Delivery Guarantee") + .description("Specifies the requirement for guaranteeing that a message is sent to Kafka") + .required(true) + .expressionLanguageSupported(false) + .allowableValues(DELIVERY_BEST_EFFORT, DELIVERY_ONE_NODE, DELIVERY_REPLICATED) + .defaultValue(DELIVERY_BEST_EFFORT.getValue()) + .build(); public static final PropertyDescriptor MESSAGE_DELIMITER = new PropertyDescriptor.Builder() - .name("Message Delimiter") - .description("Specifies the delimiter to use for splitting apart multiple messages within a single FlowFile. " - + "If not specified, the entire content of the FlowFile will be used as a single message. " - + "If specified, the contents of the FlowFile will be split on this delimiter and each section " - + "sent as a separate Kafka message.") - .required(false) - .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) - .expressionLanguageSupported(true) - .build(); + .name("Message Delimiter") + .description("Specifies the delimiter to use for splitting apart multiple messages within a single FlowFile. " + + "If not specified, the entire content of the FlowFile will be used as a single message. " + + "If specified, the contents of the FlowFile will be split on this delimiter and each section " + + "sent as a separate Kafka message.") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(true) + .build(); public static final PropertyDescriptor MAX_BUFFER_SIZE = new PropertyDescriptor.Builder() - .name("Max Buffer Size") - .description("The maximum amount of data to buffer in memory before sending to Kafka") - .required(true) - .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) - .expressionLanguageSupported(false) - .defaultValue("1 MB") - .build(); + .name("Max Buffer Size") + .description("The maximum amount of data to buffer in memory before sending to Kafka") + .required(true) + .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) + .expressionLanguageSupported(false) + .defaultValue("1 MB") + .build(); public static final PropertyDescriptor TIMEOUT = new PropertyDescriptor.Builder() - .name("Communications Timeout") - .description("The amount of time to wait for a response from Kafka before determining that there is a communications error") - .required(true) - .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) - .expressionLanguageSupported(false) - .defaultValue("30 secs") - .build(); + .name("Communications Timeout") + .description("The amount of time to wait for a response from Kafka before determining that there is a communications error") + .required(true) + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .defaultValue("30 secs") + .build(); public static final PropertyDescriptor CLIENT_NAME = new PropertyDescriptor.Builder() - .name("Client Name") - .description("Client Name to use when communicating with Kafka") - .required(true) - .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) - .expressionLanguageSupported(false) - .build(); + .name("Client Name") + .description("Client Name to use when communicating with Kafka") + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + public static final PropertyDescriptor PRODUCER_TYPE = new PropertyDescriptor.Builder() + .name("Producer Type") + .description("This parameter specifies whether the messages are sent asynchronously in a background thread.") + .required(true) + .allowableValues(PRODUCTER_TYPE_SYNCHRONOUS, PRODUCTER_TYPE_ASYNCHRONOUS) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(false) + .defaultValue(PRODUCTER_TYPE_SYNCHRONOUS.getValue()) + .build(); + public static final PropertyDescriptor BATCH_NUM_MESSAGES = new PropertyDescriptor.Builder() + .name("Async Batch Size") + .description("Used only if Producer Type is set to \"" + PRODUCTER_TYPE_ASYNCHRONOUS.getDisplayName() + "\"." + + " The number of messages to send in one batch when using " + PRODUCTER_TYPE_ASYNCHRONOUS.getDisplayName() + " mode." + + " The producer will wait until either this number of messages are ready" + + " to send or \"Queue Buffering Max Time\" is reached.") + .required(true) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("200") + .build(); + public static final PropertyDescriptor QUEUE_BUFFERING_MAX = new PropertyDescriptor.Builder() + .name("Queue Buffering Max Time") + .description("Used only if Producer Type is set to \"" + PRODUCTER_TYPE_ASYNCHRONOUS.getDisplayName() + "\"." + + " Maximum time to buffer data when using " + PRODUCTER_TYPE_ASYNCHRONOUS.getDisplayName() + " mode. For example a setting of 100 ms" + + " will try to batch together 100ms of messages to send at once. This will improve" + + " throughput but adds message delivery latency due to the buffering.") + .required(true) + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .defaultValue("5 secs") + .build(); + public static final PropertyDescriptor QUEUE_BUFFERING_MAX_MESSAGES = new PropertyDescriptor.Builder() + .name("Queue Buffer Max Count") + .description("Used only if Producer Type is set to \"" + PRODUCTER_TYPE_ASYNCHRONOUS.getDisplayName() + "\"." + + " The maximum number of unsent messages that can be queued up in the producer when" + + " using " + PRODUCTER_TYPE_ASYNCHRONOUS.getDisplayName() + " mode before either the producer must be blocked or data must be dropped.") + .required(true) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("10000") + .build(); + public static final PropertyDescriptor QUEUE_ENQUEUE_TIMEOUT = new PropertyDescriptor.Builder() + .name("Queue Enqueue Timeout") + .description("Used only if Producer Type is set to \"" + PRODUCTER_TYPE_ASYNCHRONOUS.getDisplayName() + "\"." + + " The amount of time to block before dropping messages when running in " + + PRODUCTER_TYPE_ASYNCHRONOUS.getDisplayName() + " mode" + + " and the buffer has reached the \"Queue Buffer Max Count\". If set to 0, events will" + + " be enqueued immediately or dropped if the queue is full (the producer send call will" + + " never block). If not set, the producer will block indefinitely and never willingly" + + " drop a send.") + .required(false) + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .build(); + public static final PropertyDescriptor COMPRESSION_CODEC = new PropertyDescriptor.Builder() + .name("Compression Codec") + .description("This parameter allows you to specify the compression codec for all" + + " data generated by this producer.") + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .allowableValues(COMPRESSION_CODEC_NONE, COMPRESSION_CODEC_GZIP, COMPRESSION_CODEC_SNAPPY) + .defaultValue(COMPRESSION_CODEC_NONE.getValue()) + .build(); + public static final PropertyDescriptor COMPRESSED_TOPICS = new PropertyDescriptor.Builder() + .name("Compressed Topics") + .description("This parameter allows you to set whether compression should be turned on" + + " for particular topics. If the compression codec is anything other than" + + " \"" + COMPRESSION_CODEC_NONE.getDisplayName() + "\", enable compression only for specified topics if any." + + " If the list of compressed topics is empty, then enable the specified" + + " compression codec for all topics. If the compression codec is " + COMPRESSION_CODEC_NONE.getDisplayName() + "," + + " compression is disabled for all topics") + .required(false) + .build(); public static final Relationship REL_SUCCESS = new Relationship.Builder() - .name("success") - .description("Any FlowFile that is successfully sent to Kafka will be routed to this Relationship") - .build(); + .name("success") + .description("Any FlowFile that is successfully sent to Kafka will be routed to this Relationship") + .build(); public static final Relationship REL_FAILURE = new Relationship.Builder() - .name("failure") - .description("Any FlowFile that cannot be sent to Kafka will be routed to this Relationship") - .build(); + .name("failure") + .description("Any FlowFile that cannot be sent to Kafka will be routed to this Relationship") + .build(); private final BlockingQueue> producers = new LinkedBlockingQueue<>(); @Override protected List getSupportedPropertyDescriptors() { final PropertyDescriptor clientName = new PropertyDescriptor.Builder() - .fromPropertyDescriptor(CLIENT_NAME) - .defaultValue("NiFi-" + getIdentifier()) - .build(); + .fromPropertyDescriptor(CLIENT_NAME) + .defaultValue("NiFi-" + getIdentifier()) + .build(); final List props = new ArrayList<>(); props.add(SEED_BROKERS); @@ -163,10 +262,32 @@ public class PutKafka extends AbstractProcessor { props.add(MESSAGE_DELIMITER); props.add(MAX_BUFFER_SIZE); props.add(TIMEOUT); + props.add(PRODUCER_TYPE); + props.add(BATCH_NUM_MESSAGES); + props.add(QUEUE_BUFFERING_MAX_MESSAGES); + props.add(QUEUE_BUFFERING_MAX); + props.add(QUEUE_ENQUEUE_TIMEOUT); + props.add(COMPRESSION_CODEC); + props.add(COMPRESSED_TOPICS); props.add(clientName); return props; } + @Override + public Collection customValidate(final ValidationContext context) { + final List errors = new ArrayList<>(super.customValidate(context)); + + final Integer batchMessages = context.getProperty(BATCH_NUM_MESSAGES).asInteger(); + final Integer bufferMaxMessages = context.getProperty(QUEUE_BUFFERING_MAX_MESSAGES).asInteger(); + + if (batchMessages > bufferMaxMessages) { + errors.add(new ValidationResult.Builder().subject("Batch Size, Queue Buffer").valid(false) + .explanation("Batch Size (" + batchMessages + ") must be equal to or less than the Queue Buffer Max Count (" + bufferMaxMessages + ")").build()); + } + + return errors; + } + @Override public Set getRelationships() { final Set relationships = new HashSet<>(1); @@ -194,7 +315,27 @@ public class PutKafka extends AbstractProcessor { properties.setProperty("request.timeout.ms", String.valueOf(context.getProperty(TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).longValue())); properties.setProperty("message.send.max.retries", "1"); - properties.setProperty("producer.type", "sync"); + properties.setProperty("producer.type", context.getProperty(PRODUCER_TYPE).getValue()); + properties.setProperty("batch.num.messages", context.getProperty(BATCH_NUM_MESSAGES).getValue()); + + final Long queueBufferingMillis = context.getProperty(QUEUE_BUFFERING_MAX).asTimePeriod(TimeUnit.MILLISECONDS); + if (queueBufferingMillis != null) { + properties.setProperty("queue.buffering.max.ms", String.valueOf(queueBufferingMillis)); + } + properties.setProperty("queue.buffering.max.messages", context.getProperty(QUEUE_BUFFERING_MAX_MESSAGES).getValue()); + + final Long queueEnqueueTimeoutMillis = context.getProperty(QUEUE_ENQUEUE_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS); + if (queueEnqueueTimeoutMillis != null) { + properties.setProperty("queue.enqueue.timeout.ms", String.valueOf(queueEnqueueTimeoutMillis)); + } + + final String compressionCodec = context.getProperty(COMPRESSION_CODEC).getValue(); + properties.setProperty("compression.codec", compressionCodec); + + final String compressedTopics = context.getProperty(COMPRESSED_TOPICS).getValue(); + if (compressedTopics != null) { + properties.setProperty("compressed.topics", compressedTopics); + } return new ProducerConfig(properties); } @@ -204,7 +345,7 @@ public class PutKafka extends AbstractProcessor { } private Producer borrowProducer(final ProcessContext context) { - Producer producer = producers.poll(); + final Producer producer = producers.poll(); return producer == null ? createProducer(context) : producer; } @@ -214,7 +355,7 @@ public class PutKafka extends AbstractProcessor { @Override public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { - FlowFile flowFile = session.get(); + final FlowFile flowFile = session.get(); if (flowFile == null) { return; } @@ -222,7 +363,7 @@ public class PutKafka extends AbstractProcessor { final long start = System.nanoTime(); final String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue(); final String key = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue(); - final byte[] keyBytes = (key == null) ? null : key.getBytes(StandardCharsets.UTF_8); + final byte[] keyBytes = key == null ? null : key.getBytes(StandardCharsets.UTF_8); String delimiter = context.getProperty(MESSAGE_DELIMITER).evaluateAttributeExpressions(flowFile).getValue(); if (delimiter != null) { delimiter = delimiter.replace("\\n", "\n").replace("\\r", "\r").replace("\\t", "\t"); @@ -255,9 +396,9 @@ public class PutKafka extends AbstractProcessor { session.getProvenanceReporter().send(flowFile, "kafka://" + topic); session.transfer(flowFile, REL_SUCCESS); - getLogger().info("Successfully sent {} to Kafka in {} millis", new Object[]{flowFile, TimeUnit.NANOSECONDS.toMillis(nanos)}); + getLogger().info("Successfully sent {} to Kafka in {} millis", new Object[] { flowFile, TimeUnit.NANOSECONDS.toMillis(nanos) }); } catch (final Exception e) { - getLogger().error("Failed to send {} to Kafka due to {}; routing to failure", new Object[]{flowFile, e}); + getLogger().error("Failed to send {} to Kafka due to {}; routing to failure", new Object[] { flowFile, e }); session.transfer(flowFile, REL_FAILURE); error = true; } finally { @@ -292,7 +433,7 @@ public class PutKafka extends AbstractProcessor { int nextByte; try (final InputStream bufferedIn = new BufferedInputStream(rawIn); - final ByteCountingInputStream in = new ByteCountingInputStream(bufferedIn)) { + final ByteCountingInputStream in = new ByteCountingInputStream(bufferedIn)) { // read until we're out of data. while (!streamFinished) { @@ -380,7 +521,7 @@ public class PutKafka extends AbstractProcessor { final long nanos = System.nanoTime() - start; session.getProvenanceReporter().send(flowFile, "kafka://" + topic, "Sent " + messagesSent.get() + " messages"); session.transfer(flowFile, REL_SUCCESS); - getLogger().info("Successfully sent {} messages to Kafka for {} in {} millis", new Object[]{messagesSent.get(), flowFile, TimeUnit.NANOSECONDS.toMillis(nanos)}); + getLogger().info("Successfully sent {} messages to Kafka for {} in {} millis", new Object[] { messagesSent.get(), flowFile, TimeUnit.NANOSECONDS.toMillis(nanos) }); } catch (final ProcessException pe) { error = true; @@ -390,7 +531,7 @@ public class PutKafka extends AbstractProcessor { final long offset = lastMessageOffset.get(); if (offset == 0L) { // all of the messages failed to send. Route FlowFile to failure - getLogger().error("Failed to send {} to Kafka due to {}; routing to fialure", new Object[]{flowFile, pe.getCause()}); + getLogger().error("Failed to send {} to Kafka due to {}; routing to fialure", new Object[] { flowFile, pe.getCause() }); session.transfer(flowFile, REL_FAILURE); } else { // Some of the messages were sent successfully. We want to split off the successful messages from the failed messages. @@ -398,8 +539,8 @@ public class PutKafka extends AbstractProcessor { final FlowFile failedMessages = session.clone(flowFile, offset, flowFile.getSize() - offset); getLogger().error("Successfully sent {} of the messages from {} but then failed to send the rest. Original FlowFile split into" - + " two: {} routed to 'success', {} routed to 'failure'. Failure was due to {}", new Object[]{ - messagesSent.get(), flowFile, successfulMessages, failedMessages, pe.getCause()}); + + " two: {} routed to 'success', {} routed to 'failure'. Failure was due to {}", new Object[] { + messagesSent.get(), flowFile, successfulMessages, failedMessages, pe.getCause() }); session.transfer(successfulMessages, REL_SUCCESS); session.transfer(failedMessages, REL_FAILURE); diff --git a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestPutKafka.java b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestPutKafka.java index 8dbc4d0403..750d40691e 100644 --- a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestPutKafka.java +++ b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestPutKafka.java @@ -28,6 +28,7 @@ import java.util.Map; import kafka.common.FailedToSendMessageException; import kafka.javaapi.producer.Producer; +import kafka.message.CompressionCodec; import kafka.producer.KeyedMessage; import kafka.producer.ProducerConfig; @@ -41,6 +42,8 @@ import org.apache.nifi.util.TestRunners; import org.junit.Ignore; import org.junit.Test; +import scala.collection.Seq; + public class TestPutKafka { @Test @@ -191,6 +194,24 @@ public class TestPutKafka { runner.setProperty(PutKafka.TIMEOUT, "3 secs"); runner.setProperty(PutKafka.DELIVERY_GUARANTEE, PutKafka.DELIVERY_REPLICATED.getValue()); + keyValuePutExecute(runner); + } + + @Test + @Ignore("Intended only for local testing; requires an actual running instance of Kafka & ZooKeeper...") + public void testKeyValuePutAsync() { + final TestRunner runner = TestRunners.newTestRunner(PutKafka.class); + runner.setProperty(PutKafka.SEED_BROKERS, "192.168.0.101:9092"); + runner.setProperty(PutKafka.TOPIC, "${kafka.topic}"); + runner.setProperty(PutKafka.KEY, "${kafka.key}"); + runner.setProperty(PutKafka.TIMEOUT, "3 secs"); + runner.setProperty(PutKafka.PRODUCER_TYPE, PutKafka.PRODUCTER_TYPE_ASYNCHRONOUS.getValue()); + runner.setProperty(PutKafka.DELIVERY_GUARANTEE, PutKafka.DELIVERY_REPLICATED.getValue()); + + keyValuePutExecute(runner); + } + + private void keyValuePutExecute(final TestRunner runner) { final Map attributes = new HashMap<>(); attributes.put("kafka.topic", "test"); attributes.put("kafka.key", "key3"); @@ -210,6 +231,140 @@ public class TestPutKafka { assertTrue(Arrays.equals(data, mff.toByteArray())); } + @Test + public void testProducerConfigDefault() { + + final TestableProcessor processor = new TestableProcessor(); + final TestRunner runner = TestRunners.newTestRunner(processor); + + runner.setProperty(PutKafka.TOPIC, "topic1"); + runner.setProperty(PutKafka.KEY, "key1"); + runner.setProperty(PutKafka.SEED_BROKERS, "localhost:1234"); + runner.setProperty(PutKafka.MESSAGE_DELIMITER, "\\n"); + + final ProcessContext context = runner.getProcessContext(); + final ProducerConfig config = processor.createConfig(context); + + // Check the codec + final CompressionCodec codec = config.compressionCodec(); + assertTrue(codec instanceof kafka.message.NoCompressionCodec$); + + // Check compressed topics + final Seq compressedTopics = config.compressedTopics(); + assertEquals(0, compressedTopics.size()); + + // Check the producer type + final String actualProducerType = config.producerType(); + assertEquals(PutKafka.PRODUCER_TYPE.getDefaultValue(), actualProducerType); + + } + + @Test + public void testProducerConfigAsyncWithCompression() { + + final TestableProcessor processor = new TestableProcessor(); + final TestRunner runner = TestRunners.newTestRunner(processor); + + runner.setProperty(PutKafka.TOPIC, "topic1"); + runner.setProperty(PutKafka.KEY, "key1"); + runner.setProperty(PutKafka.SEED_BROKERS, "localhost:1234"); + runner.setProperty(PutKafka.MESSAGE_DELIMITER, "\\n"); + runner.setProperty(PutKafka.PRODUCER_TYPE, PutKafka.PRODUCTER_TYPE_ASYNCHRONOUS.getValue()); + runner.setProperty(PutKafka.COMPRESSION_CODEC, PutKafka.COMPRESSION_CODEC_SNAPPY.getValue()); + runner.setProperty(PutKafka.COMPRESSED_TOPICS, "topic01,topic02,topic03"); + + final ProcessContext context = runner.getProcessContext(); + final ProducerConfig config = processor.createConfig(context); + + // Check that the codec is snappy + final CompressionCodec codec = config.compressionCodec(); + assertTrue(codec instanceof kafka.message.SnappyCompressionCodec$); + + // Check compressed topics + final Seq compressedTopics = config.compressedTopics(); + assertEquals(3, compressedTopics.size()); + assertTrue(compressedTopics.contains("topic01")); + assertTrue(compressedTopics.contains("topic02")); + assertTrue(compressedTopics.contains("topic03")); + + // Check the producer type + final String actualProducerType = config.producerType(); + assertEquals("async", actualProducerType); + + } + + @Test + public void testProducerConfigAsyncQueueThresholds() { + + final TestableProcessor processor = new TestableProcessor(); + final TestRunner runner = TestRunners.newTestRunner(processor); + + runner.setProperty(PutKafka.TOPIC, "topic1"); + runner.setProperty(PutKafka.KEY, "key1"); + runner.setProperty(PutKafka.SEED_BROKERS, "localhost:1234"); + runner.setProperty(PutKafka.MESSAGE_DELIMITER, "\\n"); + runner.setProperty(PutKafka.PRODUCER_TYPE, PutKafka.PRODUCTER_TYPE_ASYNCHRONOUS.getValue()); + runner.setProperty(PutKafka.QUEUE_BUFFERING_MAX, "7 secs"); + runner.setProperty(PutKafka.QUEUE_BUFFERING_MAX_MESSAGES, "535"); + runner.setProperty(PutKafka.QUEUE_ENQUEUE_TIMEOUT, "200 ms"); + + final ProcessContext context = runner.getProcessContext(); + final ProducerConfig config = processor.createConfig(context); + + // Check that the queue thresholds were properly translated + assertEquals(7000, config.queueBufferingMaxMs()); + assertEquals(535, config.queueBufferingMaxMessages()); + assertEquals(200, config.queueEnqueueTimeoutMs()); + + // Check the producer type + final String actualProducerType = config.producerType(); + assertEquals("async", actualProducerType); + + } + + @Test + public void testProducerConfigInvalidBatchSize() { + + final TestableProcessor processor = new TestableProcessor(); + final TestRunner runner = TestRunners.newTestRunner(processor); + + runner.setProperty(PutKafka.TOPIC, "topic1"); + runner.setProperty(PutKafka.KEY, "key1"); + runner.setProperty(PutKafka.SEED_BROKERS, "localhost:1234"); + runner.setProperty(PutKafka.MESSAGE_DELIMITER, "\\n"); + runner.setProperty(PutKafka.PRODUCER_TYPE, PutKafka.PRODUCTER_TYPE_ASYNCHRONOUS.getValue()); + runner.setProperty(PutKafka.BATCH_NUM_MESSAGES, "200"); + runner.setProperty(PutKafka.QUEUE_BUFFERING_MAX_MESSAGES, "100"); + + runner.assertNotValid(); + + } + + @Test + public void testProducerConfigAsyncDefaultEnqueueTimeout() { + + final TestableProcessor processor = new TestableProcessor(); + final TestRunner runner = TestRunners.newTestRunner(processor); + + runner.setProperty(PutKafka.TOPIC, "topic1"); + runner.setProperty(PutKafka.KEY, "key1"); + runner.setProperty(PutKafka.SEED_BROKERS, "localhost:1234"); + runner.setProperty(PutKafka.MESSAGE_DELIMITER, "\\n"); + runner.setProperty(PutKafka.PRODUCER_TYPE, PutKafka.PRODUCTER_TYPE_ASYNCHRONOUS.getValue()); + // Do not set QUEUE_ENQUEUE_TIMEOUT + + final ProcessContext context = runner.getProcessContext(); + final ProducerConfig config = processor.createConfig(context); + + // Check that the enqueue timeout defaults to -1 + assertEquals(-1, config.queueEnqueueTimeoutMs()); + + // Check the producer type + final String actualProducerType = config.producerType(); + assertEquals("async", actualProducerType); + + } + private static class TestableProcessor extends PutKafka { private MockProducer producer; @@ -236,6 +391,14 @@ public class TestPutKafka { public MockProducer getProducer() { return producer; } + + /** + * Exposed for test verification + */ + @Override + public ProducerConfig createConfig(final ProcessContext context) { + return super.createConfig(context); + } } private static class MockProducer extends Producer { diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/IndexConfiguration.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/IndexConfiguration.java index 9ea793daf9..4e808111d5 100644 --- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/IndexConfiguration.java +++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/IndexConfiguration.java @@ -85,7 +85,7 @@ public class IndexConfiguration { } private Long getFirstEntryTime(final File provenanceLogFile) { - try (final RecordReader reader = RecordReaders.newRecordReader(provenanceLogFile, null)) { + try (final RecordReader reader = RecordReaders.newRecordReader(provenanceLogFile, null, Integer.MAX_VALUE)) { final StandardProvenanceEventRecord firstRecord = reader.nextRecord(); if (firstRecord == null) { return provenanceLogFile.lastModified(); diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java index 5da5d6fe42..81d883a74e 100644 --- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java +++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java @@ -134,6 +134,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository private final IndexManager indexManager; private final boolean alwaysSync; private final int rolloverCheckMillis; + private final int maxAttributeChars; private final ScheduledExecutorService scheduledExecService; private final ScheduledExecutorService rolloverExecutor; @@ -167,6 +168,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository } this.configuration = configuration; + this.maxAttributeChars = configuration.getMaxAttributeChars(); for (final File file : configuration.getStorageDirectories()) { final Path storageDirectory = file.toPath(); @@ -289,6 +291,21 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository final Boolean alwaysSync = Boolean.parseBoolean(properties.getProperty("nifi.provenance.repository.always.sync", "false")); + final int defaultMaxAttrChars = 65536; + final String maxAttrLength = properties.getProperty("nifi.provenance.repository.max.attribute.length", String.valueOf(defaultMaxAttrChars)); + int maxAttrChars; + try { + maxAttrChars = Integer.parseInt(maxAttrLength); + // must be at least 36 characters because that's the length of the uuid attribute, + // which must be kept intact + if (maxAttrChars < 36) { + maxAttrChars = 36; + logger.warn("Found max attribute length property set to " + maxAttrLength + " but minimum length is 36; using 36 instead"); + } + } catch (final Exception e) { + maxAttrChars = defaultMaxAttrChars; + } + final List searchableFields = SearchableFieldParser.extractSearchableFields(indexedFieldString, true); final List searchableAttributes = SearchableFieldParser.extractSearchableFields(indexedAttrString, false); @@ -310,6 +327,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository config.setMaxStorageCapacity(maxStorageBytes); config.setQueryThreadPoolSize(queryThreads); config.setJournalCount(journalCount); + config.setMaxAttributeChars(maxAttrChars); if (shardSize != null) { config.setDesiredIndexSize(DataUnit.parseDataSize(shardSize, DataUnit.B).longValue()); @@ -337,6 +355,14 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository return writers; } + /** + * @return the maximum number of characters that any Event attribute should contain. If the event contains + * more characters than this, the attribute may be truncated on retrieval + */ + public int getMaxAttributeCharacters() { + return maxAttributeChars; + } + @Override public StandardProvenanceEventRecord.Builder eventBuilder() { return new StandardProvenanceEventRecord.Builder(); @@ -362,7 +388,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository } for (final Path path : paths) { - try (RecordReader reader = RecordReaders.newRecordReader(path.toFile(), getAllLogFiles())) { + try (RecordReader reader = RecordReaders.newRecordReader(path.toFile(), getAllLogFiles(), maxAttributeChars)) { // if this is the first record, try to find out the block index and jump directly to // the block index. This avoids having to read through a lot of data that we don't care about // just to get to the first record that we want. @@ -377,7 +403,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository } StandardProvenanceEventRecord record; - while (records.size() < maxRecords && ((record = reader.nextRecord()) != null)) { + while (records.size() < maxRecords && (record = reader.nextRecord()) != null) { if (record.getEventId() >= firstRecordId) { records.add(record); } @@ -507,7 +533,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository if (maxIdFile != null) { // Determine the max ID in the last file. - try (final RecordReader reader = RecordReaders.newRecordReader(maxIdFile, getAllLogFiles())) { + try (final RecordReader reader = RecordReaders.newRecordReader(maxIdFile, getAllLogFiles(), maxAttributeChars)) { final long eventId = reader.getMaxEventId(); if (eventId > maxId) { maxId = eventId; @@ -571,7 +597,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository // Read the records in the last file to find its max id if (greatestMinIdFile != null) { - try (final RecordReader recordReader = RecordReaders.newRecordReader(greatestMinIdFile, Collections.emptyList())) { + try (final RecordReader recordReader = RecordReaders.newRecordReader(greatestMinIdFile, Collections. emptyList(), maxAttributeChars)) { maxId = recordReader.getMaxEventId(); } } @@ -1224,7 +1250,10 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository try { for (final File journalFile : journalFiles) { try { - readers.add(RecordReaders.newRecordReader(journalFile, null)); + // Use MAX_VALUE for number of chars because we don't want to truncate the value as we write it + // out. This allows us to later decide that we want more characters and still be able to retrieve + // the entire event. + readers.add(RecordReaders.newRecordReader(journalFile, null, Integer.MAX_VALUE)); } catch (final EOFException eof) { // there's nothing here. Skip over it. } catch (final IOException ioe) { @@ -1314,7 +1343,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository indexingAction.index(record, indexWriter, blockIndex); maxId = record.getEventId(); - latestRecords.add(record); + latestRecords.add(truncateAttributes(record)); records++; // Remove this entry from the map @@ -1383,6 +1412,39 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository return writerFile; } + private StandardProvenanceEventRecord truncateAttributes(final StandardProvenanceEventRecord original) { + boolean requireTruncation = false; + + for (final Map.Entry entry : original.getAttributes().entrySet()) { + if (entry.getValue().length() > maxAttributeChars) { + requireTruncation = true; + break; + } + } + + if (!requireTruncation) { + return original; + } + + final StandardProvenanceEventRecord.Builder builder = new StandardProvenanceEventRecord.Builder().fromEvent(original); + builder.setAttributes(truncateAttributes(original.getPreviousAttributes()), truncateAttributes(original.getUpdatedAttributes())); + final StandardProvenanceEventRecord truncated = builder.build(); + truncated.setEventId(original.getEventId()); + return truncated; + } + + private Map truncateAttributes(final Map original) { + final Map truncatedAttrs = new HashMap<>(); + for (final Map.Entry entry : original.entrySet()) { + if (entry.getValue().length() > maxAttributeChars) { + truncatedAttrs.put(entry.getKey(), entry.getValue().substring(0, maxAttributeChars)); + } else { + truncatedAttrs.put(entry.getKey(), entry.getValue()); + } + } + return truncatedAttrs; + } + @Override public List getSearchableFields() { final List searchableFields = new ArrayList<>(configuration.getSearchableFields()); @@ -1612,7 +1674,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository for (final File file : potentialFiles) { try { - reader = RecordReaders.newRecordReader(file, allLogFiles); + reader = RecordReaders.newRecordReader(file, allLogFiles, maxAttributeChars); } catch (final IOException ioe) { continue; } @@ -1788,7 +1850,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository return true; } - if (repoDirty.get() || (writtenSinceRollover > 0 && System.currentTimeMillis() > streamStartTime.get() + maxPartitionMillis)) { + if (repoDirty.get() || writtenSinceRollover > 0 && System.currentTimeMillis() > streamStartTime.get() + maxPartitionMillis) { return true; } @@ -1797,7 +1859,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository public Collection getAllLogFiles() { final SortedMap map = idToPathMap.get(); - return (map == null) ? new ArrayList() : map.values(); + return map == null ? new ArrayList() : map.values(); } private static class PathMapComparator implements Comparator { @@ -1885,7 +1947,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository @Override public void run() { try { - final IndexSearch search = new IndexSearch(PersistentProvenanceRepository.this, indexDir, indexManager); + final IndexSearch search = new IndexSearch(PersistentProvenanceRepository.this, indexDir, indexManager, maxAttributeChars); final StandardQueryResult queryResult = search.search(query, retrievalCount); submission.getResult().update(queryResult.getMatchingEvents(), queryResult.getTotalHitCount()); if (queryResult.isFinished()) { @@ -1926,7 +1988,9 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository } try { - final Set matchingRecords = LineageQuery.computeLineageForFlowFiles(PersistentProvenanceRepository.this, indexManager, indexDir, null, flowFileUuids); + final Set matchingRecords = LineageQuery.computeLineageForFlowFiles(PersistentProvenanceRepository.this, + indexManager, indexDir, null, flowFileUuids, maxAttributeChars); + final StandardLineageResult result = submission.getResult(); result.update(matchingRecords); @@ -1959,7 +2023,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository final Map.Entry entry = queryIterator.next(); final StandardQueryResult result = entry.getValue().getResult(); - if (entry.getValue().isCanceled() || (result.isFinished() && result.getExpiration().before(now))) { + if (entry.getValue().isCanceled() || result.isFinished() && result.getExpiration().before(now)) { queryIterator.remove(); } } @@ -1969,7 +2033,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository final Map.Entry entry = lineageIterator.next(); final StandardLineageResult result = entry.getValue().getResult(); - if (entry.getValue().isCanceled() || (result.isFinished() && result.getExpiration().before(now))) { + if (entry.getValue().isCanceled() || result.isFinished() && result.getExpiration().before(now)) { lineageIterator.remove(); } } diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/RepositoryConfiguration.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/RepositoryConfiguration.java index d0d147c332..381d778595 100644 --- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/RepositoryConfiguration.java +++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/RepositoryConfiguration.java @@ -34,6 +34,7 @@ public class RepositoryConfiguration { private long desiredIndexBytes = 1024L * 1024L * 500L; // 500 MB private int journalCount = 16; private int compressionBlockBytes = 1024 * 1024; + private int maxAttributeChars = 65536; private List searchableFields = new ArrayList<>(); private List searchableAttributes = new ArrayList<>(); @@ -278,4 +279,21 @@ public class RepositoryConfiguration { public void setAlwaysSync(boolean alwaysSync) { this.alwaysSync = alwaysSync; } + + /** + * @return the maximum number of characters to include in any attribute. If an attribute in a Provenance + * Event has more than this number of characters, it will be truncated when the event is retrieved. + */ + public int getMaxAttributeChars() { + return maxAttributeChars; + } + + /** + * Sets the maximum number of characters to include in any attribute. If an attribute in a Provenance + * Event has more than this number of characters, it will be truncated when it is retrieved. + */ + public void setMaxAttributeChars(int maxAttributeChars) { + this.maxAttributeChars = maxAttributeChars; + } + } diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordReader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordReader.java index ca0d5edd26..09391072c6 100644 --- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordReader.java +++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordReader.java @@ -47,18 +47,20 @@ public class StandardRecordReader implements RecordReader { private final boolean compressed; private final TocReader tocReader; private final int headerLength; + private final int maxAttributeChars; private DataInputStream dis; private ByteCountingInputStream byteCountingIn; - public StandardRecordReader(final InputStream in, final String filename) throws IOException { - this(in, filename, null); + public StandardRecordReader(final InputStream in, final String filename, final int maxAttributeChars) throws IOException { + this(in, filename, null, maxAttributeChars); } - public StandardRecordReader(final InputStream in, final String filename, final TocReader tocReader) throws IOException { + public StandardRecordReader(final InputStream in, final String filename, final TocReader tocReader, final int maxAttributeChars) throws IOException { logger.trace("Creating RecordReader for {}", filename); rawInputStream = new ByteCountingInputStream(in); + this.maxAttributeChars = maxAttributeChars; final InputStream limitedStream; if ( tocReader == null ) { @@ -367,7 +369,8 @@ public class StandardRecordReader implements RecordReader { for (int i = 0; i < numAttributes; i++) { final String key = readLongString(dis); final String value = valueNullable ? readLongNullableString(dis) : readLongString(dis); - attrs.put(key, value); + final String truncatedValue = value.length() > maxAttributeChars ? value.substring(0, maxAttributeChars) : value; + attrs.put(key, truncatedValue); } return attrs; @@ -429,7 +432,7 @@ public class StandardRecordReader implements RecordReader { byteCountingIn.reset(); } - return (nextByte >= 0); + return nextByte >= 0; } @Override @@ -451,7 +454,7 @@ public class StandardRecordReader implements RecordReader { // committed, so we can just process the FlowFile again. } - return (lastRecord == null) ? -1L : lastRecord.getEventId(); + return lastRecord == null ? -1L : lastRecord.getEventId(); } @Override diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DeleteIndexAction.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DeleteIndexAction.java index 70bf36e108..7707352064 100644 --- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DeleteIndexAction.java +++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DeleteIndexAction.java @@ -46,9 +46,9 @@ public class DeleteIndexAction implements ExpirationAction { @Override public File execute(final File expiredFile) throws IOException { // count the number of records and determine the max event id that we are deleting. - long numDeleted = 0; + final long numDeleted = 0; long maxEventId = -1L; - try (final RecordReader reader = RecordReaders.newRecordReader(expiredFile, repository.getAllLogFiles())) { + try (final RecordReader reader = RecordReaders.newRecordReader(expiredFile, repository.getAllLogFiles(), Integer.MAX_VALUE)) { maxEventId = reader.getMaxEventId(); } catch (final IOException ioe) { logger.warn("Failed to obtain max ID present in journal file {}", expiredFile.getAbsolutePath()); @@ -65,7 +65,7 @@ public class DeleteIndexAction implements ExpirationAction { writer.deleteDocuments(term); writer.commit(); final int docsLeft = writer.numDocs(); - deleteDir = (docsLeft <= 0); + deleteDir = docsLeft <= 0; logger.debug("After expiring {}, there are {} docs left for index {}", expiredFile, docsLeft, indexingDirectory); } finally { indexManager.returnIndexWriter(indexingDirectory, writer); diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java index 02fd5c3e28..eef46281a2 100644 --- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java +++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java @@ -51,7 +51,7 @@ public class DocsReader { } public Set read(final TopDocs topDocs, final IndexReader indexReader, final Collection allProvenanceLogFiles, - final AtomicInteger retrievalCount, final int maxResults) throws IOException { + final AtomicInteger retrievalCount, final int maxResults, final int maxAttributeChars) throws IOException { if (retrievalCount.get() >= maxResults) { return Collections.emptySet(); } @@ -68,7 +68,7 @@ public class DocsReader { final long readDocuments = System.nanoTime() - start; logger.debug("Reading {} Lucene Documents took {} millis", docs.size(), TimeUnit.NANOSECONDS.toMillis(readDocuments)); - return read(docs, allProvenanceLogFiles, retrievalCount, maxResults); + return read(docs, allProvenanceLogFiles, retrievalCount, maxResults, maxAttributeChars); } @@ -108,7 +108,8 @@ public class DocsReader { } - public Set read(final List docs, final Collection allProvenanceLogFiles, final AtomicInteger retrievalCount, final int maxResults) throws IOException { + public Set read(final List docs, final Collection allProvenanceLogFiles, + final AtomicInteger retrievalCount, final int maxResults, final int maxAttributeChars) throws IOException { if (retrievalCount.get() >= maxResults) { return Collections.emptySet(); } @@ -161,7 +162,7 @@ public class DocsReader { for (final File file : potentialFiles) { try { - reader = RecordReaders.newRecordReader(file, allProvenanceLogFiles); + reader = RecordReaders.newRecordReader(file, allProvenanceLogFiles, maxAttributeChars); matchingRecords.add(getRecord(d, reader)); if ( retrievalCount.incrementAndGet() >= maxResults ) { diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java index 53869f4563..c9bb238bed 100644 --- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java +++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java @@ -39,11 +39,13 @@ public class IndexSearch { private final PersistentProvenanceRepository repository; private final File indexDirectory; private final IndexManager indexManager; + private final int maxAttributeChars; - public IndexSearch(final PersistentProvenanceRepository repo, final File indexDirectory, final IndexManager indexManager) { + public IndexSearch(final PersistentProvenanceRepository repo, final File indexDirectory, final IndexManager indexManager, final int maxAttributeChars) { this.repository = repo; this.indexDirectory = indexDirectory; this.indexManager = indexManager; + this.maxAttributeChars = maxAttributeChars; } public StandardQueryResult search(final org.apache.nifi.provenance.search.Query provenanceQuery, final AtomicInteger retrievedCount) throws IOException { @@ -82,7 +84,8 @@ public class IndexSearch { } final DocsReader docsReader = new DocsReader(repository.getConfiguration().getStorageDirectories()); - matchingRecords = docsReader.read(topDocs, searcher.getIndexReader(), repository.getAllLogFiles(), retrievedCount, provenanceQuery.getMaxResults()); + matchingRecords = docsReader.read(topDocs, searcher.getIndexReader(), repository.getAllLogFiles(), retrievedCount, + provenanceQuery.getMaxResults(), maxAttributeChars); final long readRecordsNanos = System.nanoTime() - finishSearch; logger.debug("Reading {} records took {} millis for {}", matchingRecords.size(), TimeUnit.NANOSECONDS.toMillis(readRecordsNanos), this); diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java index 502068bea5..e9e6e63462 100644 --- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java +++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java @@ -46,7 +46,7 @@ public class LineageQuery { private static final Logger logger = LoggerFactory.getLogger(LineageQuery.class); public static Set computeLineageForFlowFiles(final PersistentProvenanceRepository repo, final IndexManager indexManager, final File indexDirectory, - final String lineageIdentifier, final Collection flowFileUuids) throws IOException { + final String lineageIdentifier, final Collection flowFileUuids, final int maxAttributeChars) throws IOException { if (requireNonNull(flowFileUuids).size() > MAX_LINEAGE_UUIDS) { throw new IllegalArgumentException(String.format("Cannot compute lineage for more than %s FlowFiles. This lineage contains %s.", MAX_LINEAGE_UUIDS, flowFileUuids.size())); } @@ -94,7 +94,9 @@ public class LineageQuery { final long searchEnd = System.nanoTime(); final DocsReader docsReader = new DocsReader(repo.getConfiguration().getStorageDirectories()); - final Set recs = docsReader.read(uuidQueryTopDocs, searcher.getIndexReader(), repo.getAllLogFiles(), new AtomicInteger(0), Integer.MAX_VALUE); + final Set recs = docsReader.read(uuidQueryTopDocs, searcher.getIndexReader(), repo.getAllLogFiles(), + new AtomicInteger(0), Integer.MAX_VALUE, maxAttributeChars); + final long readDocsEnd = System.nanoTime(); logger.debug("Finished Lineage Query against {}; Lucene search took {} millis, reading records took {} millis", indexDirectory, TimeUnit.NANOSECONDS.toMillis(searchEnd - searchStart), TimeUnit.NANOSECONDS.toMillis(readDocsEnd - searchEnd)); diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordReaders.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordReaders.java index cab5e6f250..7889cd6952 100644 --- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordReaders.java +++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordReaders.java @@ -32,7 +32,18 @@ import org.apache.nifi.provenance.toc.TocUtil; public class RecordReaders { - public static RecordReader newRecordReader(File file, final Collection provenanceLogFiles) throws IOException { + /** + * Creates a new Record Reader that is capable of reading Provenance Event Journals + * + * @param file the Provenance Event Journal to read data from + * @param provenanceLogFiles collection of all provenance journal files + * @param maxAttributeChars the maximum number of characters to retrieve for any one attribute. This allows us to avoid + * issues where a FlowFile has an extremely large attribute and reading events + * for that FlowFile results in loading that attribute into memory many times, exhausting the Java Heap + * @return a Record Reader capable of reading Provenance Event Journals + * @throws IOException if unable to create a Record Reader for the given file + */ + public static RecordReader newRecordReader(File file, final Collection provenanceLogFiles, final int maxAttributeChars) throws IOException { final File originalFile = file; InputStream fis = null; @@ -92,9 +103,9 @@ public class RecordReaders { final File tocFile = TocUtil.getTocFile(file); if ( tocFile.exists() ) { final TocReader tocReader = new StandardTocReader(tocFile); - return new StandardRecordReader(fis, filename, tocReader); + return new StandardRecordReader(fis, filename, tocReader, maxAttributeChars); } else { - return new StandardRecordReader(fis, filename); + return new StandardRecordReader(fis, filename, maxAttributeChars); } } catch (final IOException ioe) { if ( fis != null ) { diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java index 7d97bcdd5d..16f0312839 100644 --- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java +++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java @@ -252,7 +252,7 @@ public class TestPersistentProvenanceRepository { assertEquals(10, recoveredRecords.size()); for (int i = 0; i < 10; i++) { final ProvenanceEventRecord recovered = recoveredRecords.get(i); - assertEquals((long) i, recovered.getEventId()); + assertEquals(i, recovered.getEventId()); assertEquals("nifi://unit-test", recovered.getTransitUri()); assertEquals(ProvenanceEventType.RECEIVE, recovered.getEventType()); assertEquals(attributes, recovered.getAttributes()); @@ -283,7 +283,7 @@ public class TestPersistentProvenanceRepository { builder.fromFlowFile(createFlowFile(3L, 3000L, attributes)); builder.setComponentId("1234"); builder.setComponentType("dummy processor"); - ProvenanceEventRecord record = builder.build(); + final ProvenanceEventRecord record = builder.build(); for (int i = 0; i < 10; i++) { repo.registerEvent(record); @@ -1106,7 +1106,7 @@ public class TestPersistentProvenanceRepository { final Query q = new Query(""); q.setMaxResults(1000); - TopDocs topDocs = searcher.search(luceneQuery, 1000); + final TopDocs topDocs = searcher.search(luceneQuery, 1000); final List docs = new ArrayList<>(); for (int i = 0; i < topDocs.scoreDocs.length; i++) { @@ -1157,7 +1157,7 @@ public class TestPersistentProvenanceRepository { for (final File file : storageDir.listFiles()) { if (file.isFile()) { - try (RecordReader reader = RecordReaders.newRecordReader(file, null)) { + try (RecordReader reader = RecordReaders.newRecordReader(file, null, 2048)) { ProvenanceEventRecord r = null; while ((r = reader.nextRecord()) != null) { @@ -1169,4 +1169,35 @@ public class TestPersistentProvenanceRepository { assertEquals(10000, counter); } + + @Test + public void testTruncateAttributes() throws IOException, InterruptedException { + final RepositoryConfiguration config = createConfiguration(); + config.setMaxAttributeChars(50); + config.setMaxEventFileLife(3, TimeUnit.SECONDS); + repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); + repo.initialize(getEventReporter()); + + final Map attributes = new HashMap<>(); + attributes.put("75chars", "123456789012345678901234567890123456789012345678901234567890123456789012345"); + + final ProvenanceEventBuilder builder = new StandardProvenanceEventRecord.Builder(); + builder.setEventTime(System.currentTimeMillis()); + builder.setEventType(ProvenanceEventType.RECEIVE); + builder.setTransitUri("nifi://unit-test"); + attributes.put("uuid", "12345678-0000-0000-0000-012345678912"); + builder.fromFlowFile(createFlowFile(3L, 3000L, attributes)); + builder.setComponentId("1234"); + builder.setComponentType("dummy processor"); + + final ProvenanceEventRecord record = builder.build(); + repo.registerEvent(record); + repo.waitForRollover(); + + final ProvenanceEventRecord retrieved = repo.getEvent(0L); + assertNotNull(retrieved); + assertEquals("12345678-0000-0000-0000-012345678912", retrieved.getAttributes().get("uuid")); + assertEquals("12345678901234567890123456789012345678901234567890", retrieved.getAttributes().get("75chars")); + } + } diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestStandardRecordReaderWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestStandardRecordReaderWriter.java index f242642fbe..d9e64e55d3 100644 --- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestStandardRecordReaderWriter.java +++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestStandardRecordReaderWriter.java @@ -74,7 +74,7 @@ public class TestStandardRecordReaderWriter { final TocReader tocReader = new StandardTocReader(tocFile); try (final FileInputStream fis = new FileInputStream(journalFile); - final StandardRecordReader reader = new StandardRecordReader(fis, journalFile.getName(), tocReader)) { + final StandardRecordReader reader = new StandardRecordReader(fis, journalFile.getName(), tocReader, 2048)) { assertEquals(0, reader.getBlockIndex()); reader.skipToBlock(0); final StandardProvenanceEventRecord recovered = reader.nextRecord(); @@ -102,7 +102,7 @@ public class TestStandardRecordReaderWriter { final TocReader tocReader = new StandardTocReader(tocFile); try (final FileInputStream fis = new FileInputStream(journalFile); - final StandardRecordReader reader = new StandardRecordReader(fis, journalFile.getName(), tocReader)) { + final StandardRecordReader reader = new StandardRecordReader(fis, journalFile.getName(), tocReader, 2048)) { assertEquals(0, reader.getBlockIndex()); reader.skipToBlock(0); final StandardProvenanceEventRecord recovered = reader.nextRecord(); @@ -133,7 +133,7 @@ public class TestStandardRecordReaderWriter { final TocReader tocReader = new StandardTocReader(tocFile); try (final FileInputStream fis = new FileInputStream(journalFile); - final StandardRecordReader reader = new StandardRecordReader(fis, journalFile.getName(), tocReader)) { + final StandardRecordReader reader = new StandardRecordReader(fis, journalFile.getName(), tocReader, 2048)) { for (int i=0; i < 10; i++) { assertEquals(0, reader.getBlockIndex()); @@ -172,12 +172,12 @@ public class TestStandardRecordReaderWriter { final TocReader tocReader = new StandardTocReader(tocFile); try (final FileInputStream fis = new FileInputStream(journalFile); - final StandardRecordReader reader = new StandardRecordReader(fis, journalFile.getName(), tocReader)) { + final StandardRecordReader reader = new StandardRecordReader(fis, journalFile.getName(), tocReader, 2048)) { for (int i=0; i < 10; i++) { final StandardProvenanceEventRecord recovered = reader.nextRecord(); System.out.println(recovered); assertNotNull(recovered); - assertEquals((long) i, recovered.getEventId()); + assertEquals(i, recovered.getEventId()); assertEquals("nifi://unit-test", recovered.getTransitUri()); } diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java index 1bcd3bfea0..5a386a628b 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java @@ -60,7 +60,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled; + "Regular Expressions are entered by adding user-defined properties; " + "the name of the property maps to the Attribute Name into which the result will be placed. " + "The first capture group, if any found, will be placed into that attribute name." - + "But all catpure groups, including the matching string sequence itself will also be " + + "But all capture groups, including the matching string sequence itself will also be " + "provided at that attribute name with an index value provided." + "The value of the property must be a valid Regular Expressions with one or more capturing groups. " + "If the Regular Expression matches more than once, only the first match will be used. " @@ -69,7 +69,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled; + "and no attributes will be applied to the FlowFile.") @DynamicProperty(name = "A FlowFile attribute", value = "A Regular Expression with one or more capturing group", description = "The first capture group, if any found, will be placed into that attribute name." - + "But all catpure groups, including the matching string sequence itself will also be " + + "But all capture groups, including the matching string sequence itself will also be " + "provided at that attribute name with an index value provided.") public class ExtractText extends AbstractProcessor { 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); diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java index 2883a758e2..65f4124d55 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java @@ -82,31 +82,34 @@ import org.apache.nifi.util.ObjectHolder; + "created from FlowFiles in different connections. This processor updates the mime.type attribute as appropriate.") @ReadsAttributes({ @ReadsAttribute(attribute = "fragment.identifier", description = "Applicable only if the property is set to Defragment. " - + "All FlowFiles with the same value for this attribute will be bundled together"), - @ReadsAttribute(attribute = "fragment.index", description = "Applicable only if the property is set to Defragment. This " - + "attribute must be present on all FlowFiles with the same value for the fragment.identifier attribute and must be a unique integer " - + "between 0 and the value of the fragment.count attribute. This attribute indicates the order in which the fragments should be assembled"), + + "All FlowFiles with the same value for this attribute will be bundled together."), + @ReadsAttribute(attribute = "fragment.index", description = "Applicable only if the property is set to Defragment. " + + "This attribute indicates the order in which the fragments should be assembled. This " + + "attribute must be present on all FlowFiles when using the Defragment Merge Strategy and must be a unique (i.e., unique across all " + + "FlowFiles that have the same value for the \"fragment.identifier\" attribute) integer " + + "between 0 and the value of the fragment.count attribute. If two or more FlowFiles have the same value for the " + + "\"fragment.identifier\" attribute and the same value for the \"fragment.index\" attribute, the behavior of this Processor is undefined."), @ReadsAttribute(attribute = "fragment.count", description = "Applicable only if the property is set to Defragment. This " - + "attribute must be present on all FlowFiles with the same value for the fragment.identifier attribute. All FlowFiles in the same " - + "bundle must have the same value for this attribute. The value of this attribute indicates how many FlowFiles should be expected " - + "in the given bundle"), + + "attribute must be present on all FlowFiles with the same value for the fragment.identifier attribute. All FlowFiles in the same " + + "bundle must have the same value for this attribute. The value of this attribute indicates how many FlowFiles should be expected " + + "in the given bundle."), @ReadsAttribute(attribute = "segment.original.filename", description = "Applicable only if the property is set to Defragment. " - + "This attribute must be present on all FlowFiles with the same value for the fragment.identifier attribute. All FlowFiles in the same " - + "bundle must have the same value for this attribute. The value of this attribute will be used for the filename of the completed merged " - + "FlowFile"), + + "This attribute must be present on all FlowFiles with the same value for the fragment.identifier attribute. All FlowFiles in the same " + + "bundle must have the same value for this attribute. The value of this attribute will be used for the filename of the completed merged " + + "FlowFile."), @ReadsAttribute(attribute = "tar.permissions", description = "Applicable only if the property is set to TAR. The value of this " - + "attribute must be 3 characters; each character must be in the range 0 to 7 (inclusive) and indicates the file permissions that should " - + "be used for the FlowFile's TAR entry. If this attribute is missing or has an invalid value, the default value of 644 will be used")}) + + "attribute must be 3 characters; each character must be in the range 0 to 7 (inclusive) and indicates the file permissions that should " + + "be used for the FlowFile's TAR entry. If this attribute is missing or has an invalid value, the default value of 644 will be used") }) @WritesAttributes({ @WritesAttribute(attribute = "filename", description = "When more than 1 file is merged, the filename comes from the segment.original.filename " - + "attribute. If that attribute does not exist in the source FlowFiles, then the filename is set to the number of nanoseconds matching " - + "system time. Then a filename extension may be applied:" - + "if Merge Format is TAR, then the filename will be appended with .tar, " - + "if Merge Format is ZIP, then the filename will be appended with .zip, " - + "if Merge Format is FlowFileStream, then the filename will be appended with .pkg"), + + "attribute. If that attribute does not exist in the source FlowFiles, then the filename is set to the number of nanoseconds matching " + + "system time. Then a filename extension may be applied:" + + "if Merge Format is TAR, then the filename will be appended with .tar, " + + "if Merge Format is ZIP, then the filename will be appended with .zip, " + + "if Merge Format is FlowFileStream, then the filename will be appended with .pkg"), @WritesAttribute(attribute = "merge.count", description = "The number of FlowFiles that were merged into this bundle"), @WritesAttribute(attribute = "merge.bin.age", description = "The age of the bin, in milliseconds, when it was merged and output. Effectively " - + "this is the greatest amount of time that any FlowFile in this bundle remained waiting in this processor before it was output")}) + + "this is the greatest amount of time that any FlowFile in this bundle remained waiting in this processor before it was output") }) @SeeAlso(SegmentContent.class) public class MergeContent extends BinFiles { @@ -131,7 +134,9 @@ public class MergeContent extends BinFiles { "Defragment", "Combines fragments that are associated by attributes back into a single cohesive FlowFile. If using this strategy, all FlowFiles must " + "have the attributes , , and or alternatively (for backward compatibility " - + "purposes) , , and "); + + "purposes) , , and . All FlowFiles with the same value for \"fragment.identifier\" " + + "will be grouped together. All FlowFiles in this group must have the same value for the \"fragment.count\" attribute. All FlowFiles " + + "in this group must have a unique value for the \"fragment.index\" attribute between 0 and the value of the \"fragment.count\" attribute."); public static final AllowableValue DELIMITER_STRATEGY_FILENAME = new AllowableValue( "Filename", "Filename", "The values of Header, Footer, and Demarcator will be retrieved from the contents of a file"); @@ -307,7 +312,7 @@ public class MergeContent extends BinFiles { @Override protected Collection additionalCustomValidation(ValidationContext context) { - Collection results = new ArrayList<>(); + final Collection results = new ArrayList<>(); final String delimiterStrategy = context.getProperty(DELIMITER_STRATEGY).getValue(); if(DELIMITER_STRATEGY_FILENAME.equals(delimiterStrategy)) { @@ -353,7 +358,7 @@ public class MergeContent extends BinFiles { @Override protected String getGroupId(final ProcessContext context, final FlowFile flowFile) { final String correlationAttributeName = context.getProperty(CORRELATION_ATTRIBUTE_NAME).getValue(); - String groupId = (correlationAttributeName == null) ? null : flowFile.getAttribute(correlationAttributeName); + String groupId = correlationAttributeName == null ? null : flowFile.getAttribute(correlationAttributeName); // when MERGE_STRATEGY is Defragment and correlationAttributeName is null then bin by fragment.identifier if (groupId == null && MERGE_STRATEGY_DEFRAGMENT.equals(context.getProperty(MERGE_STRATEGY).getValue())) { @@ -442,7 +447,7 @@ public class MergeContent extends BinFiles { bundle = session.putAllAttributes(bundle, bundleAttributes); - final String inputDescription = (binCopy.size() < 10) ? binCopy.toString() : binCopy.size() + " FlowFiles"; + final String inputDescription = binCopy.size() < 10 ? binCopy.toString() : binCopy.size() + " FlowFiles"; getLogger().info("Merged {} into {}", new Object[]{inputDescription, bundle}); session.transfer(bundle, REL_MERGED); @@ -640,18 +645,18 @@ public class MergeContent extends BinFiles { } if (".".equals(path.getName(0).toString())) { - path = (path.getNameCount() == 1) ? null : path.subpath(1, path.getNameCount()); + path = path.getNameCount() == 1 ? null : path.subpath(1, path.getNameCount()); } - return (path == null) ? "" : path.toString() + "/"; + return path == null ? "" : path.toString() + "/"; } private String createFilename(final List wrappers) { if (wrappers.size() == 1) { return wrappers.get(0).getFlowFile().getAttribute(CoreAttributes.FILENAME.key()); } else { - FlowFile ff = wrappers.get(0).getFlowFile(); - String origFilename = ff.getAttribute(SEGMENT_ORIGINAL_FILENAME); + final FlowFile ff = wrappers.get(0).getFlowFile(); + final String origFilename = ff.getAttribute(SEGMENT_ORIGINAL_FILENAME); if (origFilename != null) { return origFilename; } else {