You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ma...@apache.org on 2019/06/13 15:34:29 UTC

[nifi] 01/02: NIFI-6322: Introduced EvaluationContext to store state while making evaluator tree reusable

This is an automated email from the ASF dual-hosted git repository.

markap14 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 68d05ab575767c812dbbf7507859a1a59fbba37e
Author: Frederik Petersen <fp...@abusix.com>
AuthorDate: Wed Jun 5 14:23:21 2019 +0200

    NIFI-6322: Introduced EvaluationContext to store state while making evaluator tree reusable
    
    - EvaluationContext gets passed through evalutors for each query
    - evaluator tree is reused for multiple evaluation calls
    - added extensive test cases for reusing prepared queries
    - fixed tiny bug in DelineatedAttributeEvaluator
---
 .../expression/language/CompiledExpression.java    |   2 +-
 .../nifi/attribute/expression/language/Query.java  |  11 +-
 .../expression/language/StandardPreparedQuery.java |   3 +-
 .../language/compile/ExpressionCompiler.java       |   3 +-
 .../language/evaluation/BooleanEvaluator.java      |   2 +-
 .../language/evaluation/DateEvaluator.java         |   2 +-
 .../language/evaluation/DecimalEvaluator.java      |   2 +-
 ...NumberEvaluator.java => EvaluationContext.java} |  25 ++--
 .../expression/language/evaluation/Evaluator.java  |   4 +-
 .../language/evaluation/NumberEvaluator.java       |   2 +-
 .../language/evaluation/StringEvaluator.java       |   2 +-
 .../language/evaluation/WholeNumberEvaluator.java  |   2 +-
 .../evaluation/cast/BooleanCastEvaluator.java      |   5 +-
 .../evaluation/cast/DateCastEvaluator.java         |   5 +-
 .../evaluation/cast/DecimalCastEvaluator.java      |   5 +-
 .../evaluation/cast/NumberCastEvaluator.java       |   5 +-
 .../evaluation/cast/StringCastEvaluator.java       |   5 +-
 .../evaluation/cast/WholeNumberCastEvaluator.java  |   5 +-
 .../evaluation/functions/AndEvaluator.java         |  18 +--
 .../evaluation/functions/AppendEvaluator.java      |   7 +-
 .../functions/Base64DecodeEvaluator.java           |   5 +-
 .../functions/Base64EncodeEvaluator.java           |   5 +-
 .../functions/CharSequenceTranslatorEvaluator.java |   5 +-
 .../evaluation/functions/ContainsEvaluator.java    |   7 +-
 .../evaluation/functions/DivideEvaluator.java      |   7 +-
 .../evaluation/functions/EndsWithEvaluator.java    |   7 +-
 .../evaluation/functions/EqualsEvaluator.java      |   7 +-
 .../functions/EqualsIgnoreCaseEvaluator.java       |   7 +-
 .../evaluation/functions/FindEvaluator.java        |   9 +-
 .../evaluation/functions/FormatEvaluator.java      |   9 +-
 .../evaluation/functions/FromRadixEvaluator.java   |  11 +-
 .../functions/GetDelimitedFieldEvaluator.java      |  15 ++-
 .../functions/GetStateVariableEvaluator.java       |   9 +-
 .../evaluation/functions/GreaterThanEvaluator.java |   7 +-
 .../functions/GreaterThanOrEqualEvaluator.java     |   7 +-
 .../evaluation/functions/HostnameEvaluator.java    |   3 +-
 .../language/evaluation/functions/IPEvaluator.java |   3 +-
 .../evaluation/functions/IfElseEvaluator.java      |  11 +-
 .../language/evaluation/functions/InEvaluator.java |   7 +-
 .../evaluation/functions/IndexOfEvaluator.java     |   7 +-
 .../evaluation/functions/IsEmptyEvaluator.java     |   5 +-
 .../evaluation/functions/IsNullEvaluator.java      |   5 +-
 .../evaluation/functions/JsonPathEvaluator.java    |   9 +-
 .../evaluation/functions/LastIndexOfEvaluator.java |   7 +-
 .../evaluation/functions/LengthEvaluator.java      |   5 +-
 .../evaluation/functions/LessThanEvaluator.java    |   7 +-
 .../functions/LessThanOrEqualEvaluator.java        |   7 +-
 .../evaluation/functions/MatchesEvaluator.java     |   9 +-
 .../evaluation/functions/MathEvaluator.java        |  17 +--
 .../evaluation/functions/MinusEvaluator.java       |   7 +-
 .../evaluation/functions/ModEvaluator.java         |   7 +-
 .../evaluation/functions/MultiplyEvaluator.java    |   7 +-
 .../evaluation/functions/NotEvaluator.java         |   5 +-
 .../evaluation/functions/NotNullEvaluator.java     |   5 +-
 .../evaluation/functions/NowEvaluator.java         |   3 +-
 .../functions/NumberToDateEvaluator.java           |   5 +-
 .../functions/OneUpSequenceEvaluator.java          |   3 +-
 .../language/evaluation/functions/OrEvaluator.java |  18 +--
 .../evaluation/functions/PlusEvaluator.java        |   7 +-
 .../evaluation/functions/PrependEvaluator.java     |   7 +-
 .../functions/RandomNumberGeneratorEvaluator.java  |   3 +-
 .../evaluation/functions/ReplaceAllEvaluator.java  |   9 +-
 .../functions/ReplaceEmptyEvaluator.java           |   7 +-
 .../evaluation/functions/ReplaceEvaluator.java     |   9 +-
 .../functions/ReplaceFirstEvaluator.java           |  10 +-
 .../evaluation/functions/ReplaceNullEvaluator.java |   7 +-
 .../evaluation/functions/StartsWithEvaluator.java  |   7 +-
 .../functions/StringToDateEvaluator.java           |   9 +-
 .../functions/SubstringAfterEvaluator.java         |   7 +-
 .../functions/SubstringAfterLastEvaluator.java     |   7 +-
 .../functions/SubstringBeforeEvaluator.java        |   7 +-
 .../functions/SubstringBeforeLastEvaluator.java    |   7 +-
 .../evaluation/functions/SubstringEvaluator.java   |   9 +-
 .../evaluation/functions/ThreadEvaluator.java      |   3 +-
 .../evaluation/functions/ToLowerEvaluator.java     |   5 +-
 .../evaluation/functions/ToRadixEvaluator.java     |   9 +-
 .../evaluation/functions/ToStringEvaluator.java    |   5 +-
 .../evaluation/functions/ToUpperEvaluator.java     |   5 +-
 .../evaluation/functions/TrimEvaluator.java        |   5 +-
 .../evaluation/functions/UrlDecodeEvaluator.java   |   5 +-
 .../evaluation/functions/UrlEncodeEvaluator.java   |   5 +-
 .../evaluation/functions/UuidEvaluator.java        |   3 +-
 .../literals/BooleanLiteralEvaluator.java          |   3 +-
 .../literals/DecimalLiteralEvaluator.java          |   3 +-
 .../literals/StringLiteralEvaluator.java           |   3 +-
 .../evaluation/literals/ToLiteralEvaluator.java    |   5 +-
 .../literals/WholeNumberLiteralEvaluator.java      |   3 +-
 .../language/evaluation/reduce/CountEvaluator.java |  11 +-
 .../language/evaluation/reduce/JoinEvaluator.java  |  30 +++--
 .../selection/AllAttributesEvaluator.java          |  11 +-
 .../selection/AnyAttributeEvaluator.java           |  11 +-
 .../evaluation/selection/AttributeEvaluator.java   |   5 +-
 .../selection/DelineatedAttributeEvaluator.java    |  48 ++++---
 .../evaluation/selection/MappingEvaluator.java     |  11 +-
 .../selection/MultiMatchAttributeEvaluator.java    |  32 +++--
 .../selection/MultiNamedAttributeEvaluator.java    |  30 +++--
 .../language/TestStandardPreparedQuery.java        | 146 +++++++++++++++++++++
 97 files changed, 591 insertions(+), 312 deletions(-)

diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/CompiledExpression.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/CompiledExpression.java
index 50fcb27..52fcb1f 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/CompiledExpression.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/CompiledExpression.java
@@ -55,6 +55,6 @@ public class CompiledExpression implements Expression {
 
     @Override
     public String evaluate(final Map<String, String> variables, final AttributeValueDecorator decorator, final Map<String, String> stateVariables) {
-        return Query.evaluateExpression(getTree(), expression, variables, decorator, stateVariables);
+        return Query.evaluateExpression(tree, rootEvaluator, expression, variables, decorator, stateVariables);
     }
 }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java
index e701341..d4b0314 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language;
 
 import org.antlr.runtime.tree.Tree;
 import org.apache.nifi.attribute.expression.language.compile.ExpressionCompiler;
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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.selection.AttributeEvaluator;
@@ -42,6 +43,7 @@ public class Query {
     private final Tree tree;
     private final Evaluator<?> evaluator;
     private final AtomicBoolean evaluated = new AtomicBoolean(false);
+    private final EvaluationContext context = new EvaluationContext();
 
     private Query(final String query, final Tree tree, final Evaluator<?> evaluator) {
         this.query = query;
@@ -199,9 +201,10 @@ public class Query {
         return -1;
     }
 
-    static String evaluateExpression(final Tree tree, final String queryText, final Map<String, String> valueMap, final AttributeValueDecorator decorator,
+    static String evaluateExpression(final Tree tree, Evaluator<?> rootEvaluator, final String queryText, final Map<String, String> valueMap, final AttributeValueDecorator decorator,
                                      final Map<String, String> stateVariables) throws ProcessException {
-        final Object evaluated = Query.fromTree(tree, queryText).evaluate(valueMap, stateVariables).getValue();
+        Query query = new Query(queryText, tree, rootEvaluator);
+        final Object evaluated = query.evaluate(valueMap, stateVariables).getValue();
         if (evaluated == null) {
             return null;
         }
@@ -358,9 +361,9 @@ public class Query {
         }
         if (stateMap != null) {
             AttributesAndState attributesAndState = new AttributesAndState(attributes, stateMap);
-            return evaluator.evaluate(attributesAndState);
+            return evaluator.evaluate(attributesAndState, context);
         } else {
-            return evaluator.evaluate(attributes);
+            return evaluator.evaluate(attributes, context);
         }
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java
index fc3f9b7..8d00b59 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.literals.StringLiteralEvaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.selection.AllAttributesEvaluator;
@@ -99,7 +100,7 @@ public class StandardPreparedQuery implements PreparedQuery {
                     final Evaluator<String> nameEval = attributeEval.getNameEvaluator();
 
                     if (nameEval instanceof StringLiteralEvaluator) {
-                        final String referencedVar = nameEval.evaluate(Collections.emptyMap()).getValue();
+                        final String referencedVar = nameEval.evaluate(Collections.emptyMap(), new EvaluationContext()).getValue();
                         variables.add(referencedVar);
                     }
                 } else if (evaluator instanceof AllAttributesEvaluator) {
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/ExpressionCompiler.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/ExpressionCompiler.java
index de4333c..253d40d 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/ExpressionCompiler.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/ExpressionCompiler.java
@@ -28,6 +28,7 @@ import org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionLe
 import org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser;
 import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.DateEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.cast.BooleanCastEvaluator;
@@ -947,7 +948,7 @@ public class ExpressionCompiler {
 
                 final List<String> attributeNames = new ArrayList<>();
                 for (int i = 1; i < tree.getChildCount(); i++) {  // skip the first child because that's the name of the multi-attribute function
-                    attributeNames.add(newStringLiteralEvaluator(tree.getChild(i).getText()).evaluate(null).getValue());
+                    attributeNames.add(newStringLiteralEvaluator(tree.getChild(i).getText()).evaluate(null, new EvaluationContext()).getValue());
                 }
 
                 switch (multiAttrType) {
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/BooleanEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/BooleanEvaluator.java
index 907ee95..57cb61f 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/BooleanEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/BooleanEvaluator.java
@@ -27,7 +27,7 @@ public abstract class BooleanEvaluator implements Evaluator<Boolean> {
     }
 
     @Override
-    public int getEvaluationsRemaining() {
+    public int getEvaluationsRemaining(final EvaluationContext context) {
         return 0;
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateEvaluator.java
index caf3117..dbf6023 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateEvaluator.java
@@ -29,7 +29,7 @@ public abstract class DateEvaluator implements Evaluator<Date> {
     }
 
     @Override
-    public int getEvaluationsRemaining() {
+    public int getEvaluationsRemaining(final EvaluationContext context) {
         return 0;
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DecimalEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DecimalEvaluator.java
index f41b965..33e919f 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DecimalEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DecimalEvaluator.java
@@ -27,7 +27,7 @@ public abstract class DecimalEvaluator implements Evaluator<Double> {
     }
 
     @Override
-    public int getEvaluationsRemaining() {
+    public int getEvaluationsRemaining(final EvaluationContext context) {
         return 0;
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/EvaluationContext.java
similarity index 64%
copy from nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberEvaluator.java
copy to nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/EvaluationContext.java
index 1faf533..b40b6d2 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/EvaluationContext.java
@@ -16,28 +16,19 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation;
 
-import org.apache.nifi.expression.AttributeExpression.ResultType;
+import java.util.HashMap;
+import java.util.Map;
 
-public abstract class NumberEvaluator implements Evaluator<Number> {
-    private String token;
+public class EvaluationContext {
 
-    @Override
-    public ResultType getResultType() {
-        return ResultType.NUMBER;
-    }
+    private final Map<Evaluator<?>, Object> statePerEvaluator = new HashMap<>();
 
-    @Override
-    public int getEvaluationsRemaining() {
-        return 0;
+    public <T> T getState(Evaluator<?> evaluator, Class<T> clazz) {
+        return clazz.cast(statePerEvaluator.get(evaluator));
     }
 
-    @Override
-    public String getToken() {
-        return token;
+    public void putState(Evaluator<?> evaluator, Object state) {
+        statePerEvaluator.put(evaluator, state);
     }
 
-    @Override
-    public void setToken(final String token) {
-        this.token = token;
-    }
 }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/Evaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/Evaluator.java
index 86f95a8..45252c5 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/Evaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/Evaluator.java
@@ -22,11 +22,11 @@ import org.apache.nifi.expression.AttributeExpression.ResultType;
 
 public interface Evaluator<T> {
 
-    QueryResult<T> evaluate(Map<String, String> attributes);
+    QueryResult<T> evaluate(Map<String, String> attributes, EvaluationContext context);
 
     ResultType getResultType();
 
-    int getEvaluationsRemaining();
+    int getEvaluationsRemaining(EvaluationContext context);
 
     Evaluator<?> getSubjectEvaluator();
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberEvaluator.java
index 1faf533..44bf3b5 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberEvaluator.java
@@ -27,7 +27,7 @@ public abstract class NumberEvaluator implements Evaluator<Number> {
     }
 
     @Override
-    public int getEvaluationsRemaining() {
+    public int getEvaluationsRemaining(final EvaluationContext context) {
         return 0;
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/StringEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/StringEvaluator.java
index c6f2b79..3f91fb6 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/StringEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/StringEvaluator.java
@@ -27,7 +27,7 @@ public abstract class StringEvaluator implements Evaluator<String> {
     }
 
     @Override
-    public int getEvaluationsRemaining() {
+    public int getEvaluationsRemaining(EvaluationContext context) {
         return 0;
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/WholeNumberEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/WholeNumberEvaluator.java
index a8d189f..e666c50 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/WholeNumberEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/WholeNumberEvaluator.java
@@ -27,7 +27,7 @@ public abstract class WholeNumberEvaluator implements Evaluator<Long> {
     }
 
     @Override
-    public int getEvaluationsRemaining() {
+    public int getEvaluationsRemaining(final EvaluationContext context) {
         return 0;
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/BooleanCastEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/BooleanCastEvaluator.java
index a88b72b..b97f88e 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/BooleanCastEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/BooleanCastEvaluator.java
@@ -20,6 +20,7 @@ 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.EvaluationContext;
 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;
@@ -33,8 +34,8 @@ public class BooleanCastEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final QueryResult<String> result = subjectEvaluator.evaluate(attributes);
+    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final QueryResult<String> result = subjectEvaluator.evaluate(attributes, context);
         if (result.getValue() == null) {
             return new BooleanQueryResult(null);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/DateCastEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/DateCastEvaluator.java
index c2754d3..af584c0 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/DateCastEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/DateCastEvaluator.java
@@ -26,6 +26,7 @@ import java.util.regex.Pattern;
 
 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.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
@@ -56,8 +57,8 @@ public class DateCastEvaluator extends DateEvaluator {
     }
 
     @Override
-    public QueryResult<Date> evaluate(final Map<String, String> attributes) {
-        final QueryResult<?> result = subjectEvaluator.evaluate(attributes);
+    public QueryResult<Date> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final QueryResult<?> result = subjectEvaluator.evaluate(attributes, context);
         if (result.getValue() == null) {
             return new DateQueryResult(null);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/DecimalCastEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/DecimalCastEvaluator.java
index bce54f7..61cc4fb 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/DecimalCastEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/DecimalCastEvaluator.java
@@ -19,6 +19,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.cast;
 import org.apache.nifi.attribute.expression.language.evaluation.DateQueryResult;
 import org.apache.nifi.attribute.expression.language.evaluation.DecimalEvaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.DecimalQueryResult;
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
@@ -42,8 +43,8 @@ public class DecimalCastEvaluator extends DecimalEvaluator {
     }
 
     @Override
-    public QueryResult<Double> evaluate(final Map<String, String> attributes) {
-        final QueryResult<?> result = subjectEvaluator.evaluate(attributes);
+    public QueryResult<Double> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final QueryResult<?> result = subjectEvaluator.evaluate(attributes, context);
         if (result.getValue() == null) {
             return new DecimalQueryResult(null);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/NumberCastEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/NumberCastEvaluator.java
index 00c9ef7..be09c93 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/NumberCastEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/NumberCastEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.cast;
 
 import org.apache.nifi.attribute.expression.language.evaluation.DateQueryResult;
 import org.apache.nifi.attribute.expression.language.evaluation.DecimalQueryResult;
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -42,8 +43,8 @@ public class NumberCastEvaluator extends NumberEvaluator {
     }
 
     @Override
-    public QueryResult<Number> evaluate(final Map<String, String> attributes) {
-        final QueryResult<?> result = subjectEvaluator.evaluate(attributes);
+    public QueryResult<Number> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final QueryResult<?> result = subjectEvaluator.evaluate(attributes, context);
         if (result.getValue() == null) {
             return new NumberQueryResult(null);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/StringCastEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/StringCastEvaluator.java
index 1a3732a..2555a41 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/StringCastEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/StringCastEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.cast;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -32,8 +33,8 @@ public class StringCastEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final QueryResult<?> result = subjectEvaluator.evaluate(attributes);
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final QueryResult<?> result = subjectEvaluator.evaluate(attributes, context);
         if (result.getValue() == null) {
             return new StringQueryResult(null);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/WholeNumberCastEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/WholeNumberCastEvaluator.java
index 736bf2c..16bc33a 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/WholeNumberCastEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/WholeNumberCastEvaluator.java
@@ -20,6 +20,7 @@ import java.util.Map;
 
 import org.apache.nifi.attribute.expression.language.evaluation.DateQueryResult;
 import org.apache.nifi.attribute.expression.language.evaluation.DecimalQueryResult;
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
@@ -42,8 +43,8 @@ public class WholeNumberCastEvaluator extends WholeNumberEvaluator {
     }
 
     @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
-        final QueryResult<?> result = subjectEvaluator.evaluate(attributes);
+    public QueryResult<Long> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final QueryResult<?> result = subjectEvaluator.evaluate(attributes, context);
         if (result.getValue() == null) {
             return new WholeNumberQueryResult(null);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AndEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AndEvaluator.java
index adc41da..abdd496 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AndEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AndEvaluator.java
@@ -20,6 +20,7 @@ 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.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -27,7 +28,6 @@ public class AndEvaluator extends BooleanEvaluator {
 
     private final Evaluator<Boolean> subjectEvaluator;
     private final Evaluator<Boolean> rhsEvaluator;
-    private BooleanQueryResult rhsResult;
 
     public AndEvaluator(final Evaluator<Boolean> subjectEvaluator, final Evaluator<Boolean> rhsEvaluator) {
         this.subjectEvaluator = subjectEvaluator;
@@ -35,8 +35,8 @@ public class AndEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final QueryResult<Boolean> subjectValue = subjectEvaluator.evaluate(attributes);
+    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final QueryResult<Boolean> subjectValue = subjectEvaluator.evaluate(attributes, context);
         if (subjectValue == null) {
             return new BooleanQueryResult(null);
         }
@@ -48,18 +48,20 @@ public class AndEvaluator extends BooleanEvaluator {
         // Returning previously evaluated result.
         // The same AndEvaluator can be evaluated multiple times if subjectEvaluator is IteratingEvaluator.
         // In that case, it's enough to evaluate the right hand side.
+        final BooleanQueryResult rhsResult = context.getState(this, BooleanQueryResult.class);
         if (rhsResult != null) {
             return rhsResult;
         }
 
-        final QueryResult<Boolean> rhsValue = rhsEvaluator.evaluate(attributes);
+        final QueryResult<Boolean> rhsValue = rhsEvaluator.evaluate(attributes, context);
+        BooleanQueryResult result;
         if (rhsValue == null) {
-            rhsResult = new BooleanQueryResult(false);
+            result = new BooleanQueryResult(false);
         } else {
-            rhsResult = new BooleanQueryResult(rhsValue.getValue());
+            result = new BooleanQueryResult(rhsValue.getValue());
         }
-
-        return new BooleanQueryResult(rhsValue.getValue());
+        context.putState(this, result);
+        return result;
     }
 
     @Override
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AppendEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AppendEvaluator.java
index 80f1974..8ee50a6 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AppendEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AppendEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -34,9 +35,9 @@ public class AppendEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        final String appendValue = appendEvaluator.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
+        final String appendValue = appendEvaluator.evaluate(attributes, context).getValue();
 
         final String result = (subjectValue == null ? "" : subjectValue)
                 + (appendValue == null ? "" : appendValue);
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/Base64DecodeEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/Base64DecodeEvaluator.java
index 8224f16..e599ef6 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/Base64DecodeEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/Base64DecodeEvaluator.java
@@ -20,6 +20,7 @@ import java.io.UnsupportedEncodingException;
 import java.util.Base64;
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -34,8 +35,8 @@ public class Base64DecodeEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new StringQueryResult(null);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/Base64EncodeEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/Base64EncodeEvaluator.java
index 296f686..c865c5d 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/Base64EncodeEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/Base64EncodeEvaluator.java
@@ -20,6 +20,7 @@ import java.io.UnsupportedEncodingException;
 import java.util.Base64;
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -34,8 +35,8 @@ public class Base64EncodeEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new StringQueryResult(null);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/CharSequenceTranslatorEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/CharSequenceTranslatorEvaluator.java
index 9f86fa4..5a57c67 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/CharSequenceTranslatorEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/CharSequenceTranslatorEvaluator.java
@@ -20,6 +20,7 @@ import java.util.Map;
 
 import org.apache.commons.text.StringEscapeUtils;
 import org.apache.commons.text.translate.CharSequenceTranslator;
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -75,8 +76,8 @@ public class CharSequenceTranslatorEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         return new StringQueryResult(subjectValue == null ? "" : method.translate(subjectValue));
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ContainsEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ContainsEvaluator.java
index 54170ba..78a0683 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ContainsEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ContainsEvaluator.java
@@ -20,6 +20,7 @@ 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.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -34,13 +35,13 @@ public class ContainsEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new BooleanQueryResult(false);
         }
 
-        final String searchString = search.evaluate(attributes).getValue();
+        final String searchString = search.evaluate(attributes, context).getValue();
         return new BooleanQueryResult(searchString == null ? false : subjectValue.contains(searchString));
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/DivideEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/DivideEvaluator.java
index 711ca98..76f0e7f 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/DivideEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/DivideEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -34,13 +35,13 @@ public class DivideEvaluator extends NumberEvaluator {
     }
 
     @Override
-    public QueryResult<Number> evaluate(final Map<String, String> attributes) {
-        final Number subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<Number> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final Number subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new NumberQueryResult(null);
         }
 
-        final Number divide = divideValue.evaluate(attributes).getValue();
+        final Number divide = divideValue.evaluate(attributes, context).getValue();
         if (divide == null) {
             return new NumberQueryResult(null);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EndsWithEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EndsWithEvaluator.java
index 13bc396..db7b4d6 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EndsWithEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EndsWithEvaluator.java
@@ -20,6 +20,7 @@ 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.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -34,13 +35,13 @@ public class EndsWithEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new BooleanQueryResult(false);
         }
 
-        final String searchString = search.evaluate(attributes).getValue();
+        final String searchString = search.evaluate(attributes, context).getValue();
         return new BooleanQueryResult(searchString == null ? false : subjectValue.endsWith(searchString));
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EqualsEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EqualsEvaluator.java
index ec67b0f..78227b4 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EqualsEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EqualsEvaluator.java
@@ -21,6 +21,7 @@ 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.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
@@ -36,13 +37,13 @@ public class EqualsEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final Object a = subject.evaluate(attributes).getValue();
+    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final Object a = subject.evaluate(attributes, context).getValue();
         if (a == null) {
             return new BooleanQueryResult(false);
         }
 
-        final Object b = compareTo.evaluate(attributes).getValue();
+        final Object b = compareTo.evaluate(attributes, context).getValue();
         if (b == null) {
             return new BooleanQueryResult(false);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EqualsIgnoreCaseEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EqualsIgnoreCaseEvaluator.java
index 344395b..6903c73 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EqualsIgnoreCaseEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EqualsIgnoreCaseEvaluator.java
@@ -20,6 +20,7 @@ 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.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -34,13 +35,13 @@ public class EqualsIgnoreCaseEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final Object a = subject.evaluate(attributes).getValue();
+    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final Object a = subject.evaluate(attributes, context).getValue();
         if (a == null) {
             return new BooleanQueryResult(false);
         }
 
-        final Object b = compareTo.evaluate(attributes).getValue();
+        final Object b = compareTo.evaluate(attributes, context).getValue();
         if (b == null) {
             return new BooleanQueryResult(false);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FindEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FindEvaluator.java
index fa2aaf5..22deac9 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FindEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FindEvaluator.java
@@ -21,6 +21,7 @@ import java.util.regex.Pattern;
 
 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.EvaluationContext;
 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.literals.StringLiteralEvaluator;
@@ -39,21 +40,21 @@ public class FindEvaluator extends BooleanEvaluator {
         // if the search string is a literal, we don't need to evaluate it each time; we can just
         // pre-compile it. Otherwise, it must be compiled every time.
         if (search instanceof StringLiteralEvaluator) {
-            this.compiledPattern = Pattern.compile(search.evaluate(null).getValue());
+            this.compiledPattern = Pattern.compile(search.evaluate(null, new EvaluationContext()).getValue());
         } else {
             this.compiledPattern = null;
         }
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new BooleanQueryResult(false);
         }
         final Pattern pattern;
         if (compiledPattern == null) {
-            String expression = search.evaluate(attributes).getValue();
+            String expression = search.evaluate(attributes, context).getValue();
             if (expression == null) {
                 return new BooleanQueryResult(false);
             }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FormatEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FormatEvaluator.java
index 00ce430..78ae64d 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FormatEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FormatEvaluator.java
@@ -23,6 +23,7 @@ import java.util.Map;
 import java.util.TimeZone;
 
 import org.apache.nifi.attribute.expression.language.evaluation.DateEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -41,13 +42,13 @@ public class FormatEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final Date subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final Date subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new StringQueryResult(null);
         }
 
-        final QueryResult<String> formatResult = format.evaluate(attributes);
+        final QueryResult<String> formatResult = format.evaluate(attributes, context);
         final String format = formatResult.getValue();
         if (format == null) {
             return null;
@@ -56,7 +57,7 @@ public class FormatEvaluator extends StringEvaluator {
         final SimpleDateFormat sdf = new SimpleDateFormat(format, Locale.US);
 
         if(timeZone != null) {
-            final QueryResult<String> tzResult = timeZone.evaluate(attributes);
+            final QueryResult<String> tzResult = timeZone.evaluate(attributes, context);
             final String tz = tzResult.getValue();
             if(tz != null && TimeZone.getTimeZone(tz) != null) {
                 sdf.setTimeZone(TimeZone.getTimeZone(tz));
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FromRadixEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FromRadixEvaluator.java
index 8febd2e..5369379 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FromRadixEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FromRadixEvaluator.java
@@ -16,13 +16,14 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
+import java.util.Map;
+
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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.WholeNumberEvaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.WholeNumberQueryResult;
 
-import java.util.Map;
-
 public class FromRadixEvaluator extends WholeNumberEvaluator {
 
     private final Evaluator<String> numberEvaluator;
@@ -34,13 +35,13 @@ public class FromRadixEvaluator extends WholeNumberEvaluator {
     }
 
     @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
-        final String result = numberEvaluator.evaluate(attributes).getValue();
+    public QueryResult<Long> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String result = numberEvaluator.evaluate(attributes, context).getValue();
         if (result == null) {
             return new WholeNumberQueryResult(null);
         }
 
-        final Long radix = radixEvaluator.evaluate(attributes).getValue();
+        final Long radix = radixEvaluator.evaluate(attributes, context).getValue();
         if (radix == null) {
             return new WholeNumberQueryResult(null);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GetDelimitedFieldEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GetDelimitedFieldEvaluator.java
index e23634a..b287ece 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GetDelimitedFieldEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GetDelimitedFieldEvaluator.java
@@ -19,6 +19,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -64,13 +65,13 @@ public class GetDelimitedFieldEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subject = subjectEval.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subject = subjectEval.evaluate(attributes, context).getValue();
         if (subject == null || subject.isEmpty()) {
             return new StringQueryResult("");
         }
 
-        final Long index = indexEval.evaluate(attributes).getValue();
+        final Long index = indexEval.evaluate(attributes, context).getValue();
         if (index == null) {
             throw new AttributeExpressionLanguageException("Cannot evaluate getDelimitedField function because the index (which field to obtain) was not specified");
         }
@@ -78,7 +79,7 @@ public class GetDelimitedFieldEvaluator extends StringEvaluator {
             return new StringQueryResult("");
         }
 
-        final String delimiter = delimiterEval.evaluate(attributes).getValue();
+        final String delimiter = delimiterEval.evaluate(attributes, context).getValue();
         if (delimiter == null || delimiter.isEmpty()) {
             throw new AttributeExpressionLanguageException("Cannot evaluate getDelimitedField function because the delimiter was not specified");
         } else if (delimiter.length() > 1) {
@@ -86,7 +87,7 @@ public class GetDelimitedFieldEvaluator extends StringEvaluator {
                 + "\", but only a single character is allowed.");
         }
 
-        final String quoteString = quoteCharEval.evaluate(attributes).getValue();
+        final String quoteString = quoteCharEval.evaluate(attributes, context).getValue();
         if (quoteString == null || quoteString.isEmpty()) {
             throw new AttributeExpressionLanguageException("Cannot evaluate getDelimitedField function because the quote character "
                 + "(which character is used to enclose values that contain the delimiter) was not specified");
@@ -95,7 +96,7 @@ public class GetDelimitedFieldEvaluator extends StringEvaluator {
                 + "(which character is used to enclose values that contain the delimiter) evaluated to \"" + quoteString + "\", but only a single character is allowed.");
         }
 
-        final String escapeString = escapeCharEval.evaluate(attributes).getValue();
+        final String escapeString = escapeCharEval.evaluate(attributes, context).getValue();
         if (escapeString == null || escapeString.isEmpty()) {
             throw new AttributeExpressionLanguageException("Cannot evaluate getDelimitedField function because the escape character "
                 + "(which character is used to escape the quote character or delimiter) was not specified");
@@ -104,7 +105,7 @@ public class GetDelimitedFieldEvaluator extends StringEvaluator {
                 + "(which character is used to escape the quote character or delimiter) evaluated to \"" + escapeString + "\", but only a single character is allowed.");
         }
 
-        Boolean stripChars = stripCharsEval.evaluate(attributes).getValue();
+        Boolean stripChars = stripCharsEval.evaluate(attributes, context).getValue();
         if (stripChars == null) {
             stripChars = Boolean.FALSE;
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GetStateVariableEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GetStateVariableEvaluator.java
index 8808e17..241c3a8 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GetStateVariableEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GetStateVariableEvaluator.java
@@ -17,14 +17,15 @@
 
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
+import java.util.Map;
+
 import org.apache.nifi.attribute.expression.language.AttributesAndState;
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
 
-import java.util.Map;
-
 public class GetStateVariableEvaluator extends StringEvaluator {
 
     private final Evaluator<String> subject;
@@ -34,12 +35,12 @@ public class GetStateVariableEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(Map<String, String> attributes) {
+    public QueryResult<String> evaluate(Map<String, String> attributes, final EvaluationContext context) {
         if (!(attributes instanceof AttributesAndState)){
             return new StringQueryResult(null);
         }
 
-        final String subjectValue = subject.evaluate(attributes).getValue();
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new StringQueryResult(null);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanEvaluator.java
index 26d3ea9..1bbc7d4 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanEvaluator.java
@@ -20,6 +20,7 @@ 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.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -34,13 +35,13 @@ public class GreaterThanEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final Number subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final Number subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new BooleanQueryResult(false);
         }
 
-        final Number comparisonValue = comparison.evaluate(attributes).getValue();
+        final Number comparisonValue = comparison.evaluate(attributes, context).getValue();
         if (comparisonValue == null) {
             return new BooleanQueryResult(false);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanOrEqualEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanOrEqualEvaluator.java
index 4b6eb5f..2683566 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanOrEqualEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanOrEqualEvaluator.java
@@ -20,6 +20,7 @@ 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.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -34,13 +35,13 @@ public class GreaterThanOrEqualEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final Number subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final Number subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new BooleanQueryResult(false);
         }
 
-        final Number comparisonValue = comparison.evaluate(attributes).getValue();
+        final Number comparisonValue = comparison.evaluate(attributes, context).getValue();
         if (comparisonValue == null) {
             return new BooleanQueryResult(false);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/HostnameEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/HostnameEvaluator.java
index c0b1134..4fb9913 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/HostnameEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/HostnameEvaluator.java
@@ -20,6 +20,7 @@ import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -46,7 +47,7 @@ public class HostnameEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
         return hostname;
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IPEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IPEvaluator.java
index 21f9fe8..ac3cd44 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IPEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IPEvaluator.java
@@ -20,6 +20,7 @@ import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -34,7 +35,7 @@ public class IPEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
         return ipAddress;
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IfElseEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IfElseEvaluator.java
index 79bc4a5..ff68b15 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IfElseEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IfElseEvaluator.java
@@ -16,13 +16,14 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
+import java.util.Map;
+
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
 
-import java.util.Map;
-
 public class IfElseEvaluator extends StringEvaluator {
 
     private final Evaluator<Boolean> subject;
@@ -36,12 +37,12 @@ public class IfElseEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final QueryResult<Boolean> subjectValue = subject.evaluate(attributes);
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final QueryResult<Boolean> subjectValue = subject.evaluate(attributes, context);
         if (subjectValue == null) {
             return new StringQueryResult(null);
         }
-        final String ifElseValue = (Boolean.TRUE.equals(subjectValue.getValue())) ? trueEvaluator.evaluate(attributes).getValue() : falseEvaluator.evaluate(attributes).getValue();
+        final String ifElseValue = (Boolean.TRUE.equals(subjectValue.getValue())) ? trueEvaluator.evaluate(attributes, context).getValue() : falseEvaluator.evaluate(attributes, context).getValue();
         return new StringQueryResult(ifElseValue);
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/InEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/InEvaluator.java
index 5a2516c..0b4e0ea 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/InEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/InEvaluator.java
@@ -21,6 +21,7 @@ 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.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -35,15 +36,15 @@ public class InEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new BooleanQueryResult(false);
         }
 
         boolean isInList = false;
         for (Evaluator<String> evaluator : search) {
-            final String searchString = evaluator.evaluate(attributes).getValue();
+            final String searchString = evaluator.evaluate(attributes, context).getValue();
             isInList = searchString == null ? false : subjectValue.equals(searchString);
             if(isInList) {
                 break;
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IndexOfEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IndexOfEvaluator.java
index 073570c..9fc4282 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IndexOfEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IndexOfEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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.WholeNumberEvaluator;
@@ -34,12 +35,12 @@ public class IndexOfEvaluator extends WholeNumberEvaluator {
     }
 
     @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<Long> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new WholeNumberQueryResult(-1L);
         }
-        final String indexEvalValue = indexEvaluator.evaluate(attributes).getValue();
+        final String indexEvalValue = indexEvaluator.evaluate(attributes, context).getValue();
 
         return new WholeNumberQueryResult((long) subjectValue.indexOf(indexEvalValue));
     }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IsEmptyEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IsEmptyEvaluator.java
index e6e9fc9..07e7048 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IsEmptyEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IsEmptyEvaluator.java
@@ -20,6 +20,7 @@ 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.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -32,8 +33,8 @@ public class IsEmptyEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final Object subjectValue = subjectEvaluator.evaluate(attributes).getValue();
+    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final Object subjectValue = subjectEvaluator.evaluate(attributes, context).getValue();
         return new BooleanQueryResult(subjectValue == null || subjectValue.toString().trim().isEmpty());
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IsNullEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IsNullEvaluator.java
index a9d678c..6f2657b 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IsNullEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IsNullEvaluator.java
@@ -20,6 +20,7 @@ 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.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -32,8 +33,8 @@ public class IsNullEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final Object subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final Object subjectValue = subject.evaluate(attributes, context).getValue();
         return new BooleanQueryResult(subjectValue == null);
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/JsonPathEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/JsonPathEvaluator.java
index 0ef39d1..32358fb 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/JsonPathEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/JsonPathEvaluator.java
@@ -20,6 +20,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -52,7 +53,7 @@ public class JsonPathEvaluator extends StringEvaluator {
         // time; we can just
         // pre-compile it. Otherwise, it must be compiled every time.
         if (jsonPathExp instanceof StringLiteralEvaluator) {
-            precompiledJsonPathExp = compileJsonPathExpression(jsonPathExp.evaluate(null).getValue());
+            precompiledJsonPathExp = compileJsonPathExpression(jsonPathExp.evaluate(null, new EvaluationContext()).getValue());
         } else {
             precompiledJsonPathExp = null;
         }
@@ -60,8 +61,8 @@ public class JsonPathEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null || subjectValue.length() == 0) {
             throw new  AttributeExpressionLanguageException("Subject is empty");
         }
@@ -76,7 +77,7 @@ public class JsonPathEvaluator extends StringEvaluator {
         if (precompiledJsonPathExp != null) {
             compiledJsonPath = precompiledJsonPathExp;
         } else {
-            compiledJsonPath = compileJsonPathExpression(jsonPathExp.evaluate(attributes).getValue());
+            compiledJsonPath = compileJsonPathExpression(jsonPathExp.evaluate(attributes, context).getValue());
         }
 
         Object result = null;
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LastIndexOfEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LastIndexOfEvaluator.java
index c8bb2b0..0e8e57d 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LastIndexOfEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LastIndexOfEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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.WholeNumberEvaluator;
@@ -34,12 +35,12 @@ public class LastIndexOfEvaluator extends WholeNumberEvaluator {
     }
 
     @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<Long> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new WholeNumberQueryResult(-1L);
         }
-        final String indexEvalValue = indexEvaluator.evaluate(attributes).getValue();
+        final String indexEvalValue = indexEvaluator.evaluate(attributes, context).getValue();
 
         return new WholeNumberQueryResult((long) subjectValue.lastIndexOf(indexEvalValue));
     }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LengthEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LengthEvaluator.java
index 27a70ad..b537025 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LengthEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LengthEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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.WholeNumberEvaluator;
@@ -32,8 +33,8 @@ public class LengthEvaluator extends WholeNumberEvaluator {
     }
 
     @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<Long> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         return new WholeNumberQueryResult((long) (subjectValue == null ? 0 : subjectValue.length()));
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanEvaluator.java
index 4b614fb..2dbe098 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanEvaluator.java
@@ -20,6 +20,7 @@ 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.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -34,13 +35,13 @@ public class LessThanEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final Number subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final Number subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new BooleanQueryResult(false);
         }
 
-        final Number comparisonValue = comparison.evaluate(attributes).getValue();
+        final Number comparisonValue = comparison.evaluate(attributes, context).getValue();
         if (comparisonValue == null) {
             return new BooleanQueryResult(false);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanOrEqualEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanOrEqualEvaluator.java
index eea4820..90170c7 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanOrEqualEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanOrEqualEvaluator.java
@@ -20,6 +20,7 @@ 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.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -34,13 +35,13 @@ public class LessThanOrEqualEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final Number subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final Number subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new BooleanQueryResult(false);
         }
 
-        final Number comparisonValue = comparison.evaluate(attributes).getValue();
+        final Number comparisonValue = comparison.evaluate(attributes, context).getValue();
         if (comparisonValue == null) {
             return new BooleanQueryResult(false);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MatchesEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MatchesEvaluator.java
index d238938..c70b831 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MatchesEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MatchesEvaluator.java
@@ -21,6 +21,7 @@ import java.util.regex.Pattern;
 
 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.EvaluationContext;
 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.literals.StringLiteralEvaluator;
@@ -39,21 +40,21 @@ public class MatchesEvaluator extends BooleanEvaluator {
         // if the search string is a literal, we don't need to evaluate it each time; we can just
         // pre-compile it. Otherwise, it must be compiled every time.
         if (search instanceof StringLiteralEvaluator) {
-            this.compiledPattern = Pattern.compile(search.evaluate(null).getValue());
+            this.compiledPattern = Pattern.compile(search.evaluate(null, new EvaluationContext()).getValue());
         } else {
             this.compiledPattern = null;
         }
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new BooleanQueryResult(false);
         }
         final Pattern pattern;
         if (compiledPattern == null) {
-            String expression = search.evaluate(attributes).getValue();
+            String expression = search.evaluate(attributes, context).getValue();
             if (expression == null) {
                 return new BooleanQueryResult(false);
             }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MathEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MathEvaluator.java
index 46b4109..7bb8a90 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MathEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MathEvaluator.java
@@ -16,16 +16,17 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.Map;
+
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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.exception.AttributeExpressionLanguageException;
 
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.util.Map;
-
 public class MathEvaluator extends NumberEvaluator {
 
     private final Evaluator<Number> subject;
@@ -39,15 +40,15 @@ public class MathEvaluator extends NumberEvaluator {
     }
 
     @Override
-    public QueryResult<Number> evaluate(final Map<String, String> attributes) {
-        final String methodNamedValue = methodName.evaluate(attributes).getValue();
+    public QueryResult<Number> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String methodNamedValue = methodName.evaluate(attributes, context).getValue();
         if (methodNamedValue == null) {
             return new NumberQueryResult(null);
         }
 
         final Number subjectValue;
         if(subject != null) {
-            subjectValue = subject.evaluate(attributes).getValue();
+            subjectValue = subject.evaluate(attributes, context).getValue();
             if(subjectValue == null){
                 return new NumberQueryResult(null);
             }
@@ -57,7 +58,7 @@ public class MathEvaluator extends NumberEvaluator {
 
         final Number optionalArgValue;
         if(optionalArg != null) {
-            optionalArgValue = optionalArg.evaluate(attributes).getValue();
+            optionalArgValue = optionalArg.evaluate(attributes, context).getValue();
 
             if(optionalArgValue == null) {
                 return new NumberQueryResult(null);
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MinusEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MinusEvaluator.java
index 2940252..9412983 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MinusEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MinusEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -34,13 +35,13 @@ public class MinusEvaluator extends NumberEvaluator {
     }
 
     @Override
-    public QueryResult<Number> evaluate(final Map<String, String> attributes) {
-        final Number subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<Number> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final Number subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new NumberQueryResult(null);
         }
 
-        final Number minus = minusValue.evaluate(attributes).getValue();
+        final Number minus = minusValue.evaluate(attributes, context).getValue();
         if (minus == null) {
             return new NumberQueryResult(null);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ModEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ModEvaluator.java
index eb42b3d..22f4f88 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ModEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ModEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -34,13 +35,13 @@ public class ModEvaluator extends NumberEvaluator {
     }
 
     @Override
-    public QueryResult<Number> evaluate(final Map<String, String> attributes) {
-        final Number subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<Number> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final Number subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new NumberQueryResult(null);
         }
 
-        final Number mod = modValue.evaluate(attributes).getValue();
+        final Number mod = modValue.evaluate(attributes, context).getValue();
         if (mod == null) {
             return new NumberQueryResult(null);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MultiplyEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MultiplyEvaluator.java
index 3d98ddd..664aeed 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MultiplyEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MultiplyEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -34,13 +35,13 @@ public class MultiplyEvaluator extends NumberEvaluator {
     }
 
     @Override
-    public QueryResult<Number> evaluate(final Map<String, String> attributes) {
-        final Number subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<Number> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final Number subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new NumberQueryResult(null);
         }
 
-        final Number multiply = multiplyValue.evaluate(attributes).getValue();
+        final Number multiply = multiplyValue.evaluate(attributes, context).getValue();
         if (multiply == null) {
             return new NumberQueryResult(null);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotEvaluator.java
index 2a09e33..15c30f5 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotEvaluator.java
@@ -20,6 +20,7 @@ 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.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -32,8 +33,8 @@ public class NotEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final QueryResult<Boolean> subjectValue = subjectEvaluator.evaluate(attributes);
+    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final QueryResult<Boolean> subjectValue = subjectEvaluator.evaluate(attributes, context);
         if (subjectValue == null) {
             return new BooleanQueryResult(null);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotNullEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotNullEvaluator.java
index 126fbfa..d566970 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotNullEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotNullEvaluator.java
@@ -20,6 +20,7 @@ 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.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -32,8 +33,8 @@ public class NotNullEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final Object subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final Object subjectValue = subject.evaluate(attributes, context).getValue();
         return new BooleanQueryResult(subjectValue != null);
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NowEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NowEvaluator.java
index bfde6b0..9052f09 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NowEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NowEvaluator.java
@@ -21,13 +21,14 @@ 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.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
 public class NowEvaluator extends DateEvaluator {
 
     @Override
-    public QueryResult<Date> evaluate(final Map<String, String> attributes) {
+    public QueryResult<Date> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
         return new DateQueryResult(new Date());
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NumberToDateEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NumberToDateEvaluator.java
index c4b14bb..5fc2e2e 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NumberToDateEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NumberToDateEvaluator.java
@@ -21,6 +21,7 @@ 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.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -33,8 +34,8 @@ public class NumberToDateEvaluator extends DateEvaluator {
     }
 
     @Override
-    public QueryResult<Date> evaluate(final Map<String, String> attributes) {
-        final QueryResult<Long> result = subject.evaluate(attributes);
+    public QueryResult<Date> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final QueryResult<Long> result = subject.evaluate(attributes, context);
         final Long value = result.getValue();
         if (value == null) {
             return new DateQueryResult(null);
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OneUpSequenceEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OneUpSequenceEvaluator.java
index 5c507d6..f33e2d4 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OneUpSequenceEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OneUpSequenceEvaluator.java
@@ -19,6 +19,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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.WholeNumberEvaluator;
@@ -29,7 +30,7 @@ public class OneUpSequenceEvaluator extends WholeNumberEvaluator {
     private static final AtomicLong value = new AtomicLong(0L);
 
     @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
+    public QueryResult<Long> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
         return new WholeNumberQueryResult(value.getAndIncrement());
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OrEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OrEvaluator.java
index 9c63c27..f8b9c6a 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OrEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OrEvaluator.java
@@ -20,6 +20,7 @@ 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.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -27,7 +28,6 @@ public class OrEvaluator extends BooleanEvaluator {
 
     private final Evaluator<Boolean> subjectEvaluator;
     private final Evaluator<Boolean> rhsEvaluator;
-    private BooleanQueryResult rhsResult;
 
     public OrEvaluator(final Evaluator<Boolean> subjectEvaluator, final Evaluator<Boolean> rhsEvaluator) {
         this.subjectEvaluator = subjectEvaluator;
@@ -35,8 +35,8 @@ public class OrEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final QueryResult<Boolean> subjectValue = subjectEvaluator.evaluate(attributes);
+    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final QueryResult<Boolean> subjectValue = subjectEvaluator.evaluate(attributes, context);
         if (subjectValue == null) {
             return new BooleanQueryResult(null);
         }
@@ -48,18 +48,20 @@ public class OrEvaluator extends BooleanEvaluator {
         // Returning previously evaluated result.
         // The same OrEvaluator can be evaluated multiple times if subjectEvaluator is IteratingEvaluator.
         // In that case, it's enough to evaluate the right hand side.
+        final BooleanQueryResult rhsResult = context.getState(this, BooleanQueryResult.class);
         if (rhsResult != null) {
             return rhsResult;
         }
 
-        final QueryResult<Boolean> rhsValue = rhsEvaluator.evaluate(attributes);
+        final QueryResult<Boolean> rhsValue = rhsEvaluator.evaluate(attributes, context);
+        BooleanQueryResult result;
         if (rhsValue == null) {
-            rhsResult = new BooleanQueryResult(false);
+            result = new BooleanQueryResult(false);
         } else {
-            rhsResult = new BooleanQueryResult(rhsValue.getValue());
+            result = new BooleanQueryResult(rhsValue.getValue());
         }
-
-        return rhsResult;
+        context.putState(this, result);
+        return result;
     }
 
     @Override
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.java
index be275e5..1bcba7c 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -34,13 +35,13 @@ public class PlusEvaluator extends NumberEvaluator {
     }
 
     @Override
-    public QueryResult<Number> evaluate(final Map<String, String> attributes) {
-        final Number subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<Number> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final Number subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new NumberQueryResult(null);
         }
 
-        final Number plus = plusValue.evaluate(attributes).getValue();
+        final Number plus = plusValue.evaluate(attributes, context).getValue();
         if (plus == null) {
             return new NumberQueryResult(null);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PrependEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PrependEvaluator.java
index 1053328..a7363cd 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PrependEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PrependEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -34,9 +35,9 @@ public class PrependEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        final String prependValue = prependEvaluator.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
+        final String prependValue = prependEvaluator.evaluate(attributes, context).getValue();
 
         final String result = (prependValue == null ? "" : prependValue) + (subjectValue == null ? "" : subjectValue);
         return new StringQueryResult(result);
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/RandomNumberGeneratorEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/RandomNumberGeneratorEvaluator.java
index 7272c04..023e94f 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/RandomNumberGeneratorEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/RandomNumberGeneratorEvaluator.java
@@ -19,6 +19,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 import java.util.Map;
 import java.util.Random;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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.WholeNumberEvaluator;
@@ -30,7 +31,7 @@ public class RandomNumberGeneratorEvaluator extends WholeNumberEvaluator {
 
 
     @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
+    public QueryResult<Long> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
         return new WholeNumberQueryResult(Math.abs(RNG.nextLong()));
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceAllEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceAllEvaluator.java
index b796233..ba5fdff 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceAllEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceAllEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -36,13 +37,13 @@ public class ReplaceAllEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new StringQueryResult(null);
         }
-        final String searchValue = search.evaluate(attributes).getValue();
-        final String replacementValue = replacement.evaluate(attributes).getValue();
+        final String searchValue = search.evaluate(attributes, context).getValue();
+        final String replacementValue = replacement.evaluate(attributes, context).getValue();
 
         return new StringQueryResult(subjectValue.replaceAll(searchValue, replacementValue));
     }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEmptyEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEmptyEvaluator.java
index 9c2cb6d..219ae57 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEmptyEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEmptyEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -33,12 +34,12 @@ public class ReplaceEmptyEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final QueryResult<String> subjectResult = subjectEvaluator.evaluate(attributes);
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final QueryResult<String> subjectResult = subjectEvaluator.evaluate(attributes, context);
         final String subjectValue = subjectResult.getValue();
         final boolean isEmpty = subjectValue == null || subjectValue.toString().trim().isEmpty();
         if (isEmpty) {
-            return replacementEvaluator.evaluate(attributes);
+            return replacementEvaluator.evaluate(attributes, context);
         } else {
             return subjectResult;
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEvaluator.java
index e55e963..3854104 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -36,13 +37,13 @@ public class ReplaceEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new StringQueryResult(null);
         }
-        final String searchValue = search.evaluate(attributes).getValue();
-        final String replacementValue = replacement.evaluate(attributes).getValue();
+        final String searchValue = search.evaluate(attributes, context).getValue();
+        final String replacementValue = replacement.evaluate(attributes, context).getValue();
 
         return new StringQueryResult(subjectValue.replace(searchValue, replacementValue));
     }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceFirstEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceFirstEvaluator.java
index 934357b..75c5ad1 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceFirstEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceFirstEvaluator.java
@@ -17,6 +17,8 @@
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
+
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -35,13 +37,13 @@ public class ReplaceFirstEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new StringQueryResult(null);
         }
-        final String searchValue = search.evaluate(attributes).getValue();
-        final String replacementValue = replacement.evaluate(attributes).getValue();
+        final String searchValue = search.evaluate(attributes, context).getValue();
+        final String replacementValue = replacement.evaluate(attributes, context).getValue();
 
         return new StringQueryResult(subjectValue.replaceFirst(searchValue, replacementValue));
     }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceNullEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceNullEvaluator.java
index e8487c6..7968e6e 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceNullEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceNullEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -34,9 +35,9 @@ public class ReplaceNullEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        return new StringQueryResult(subjectValue == null ? resultEvaluator.evaluate(attributes).getValue() : subjectValue);
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
+        return new StringQueryResult(subjectValue == null ? resultEvaluator.evaluate(attributes, context).getValue() : subjectValue);
     }
 
     @Override
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StartsWithEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StartsWithEvaluator.java
index 6a43550..55c3556 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StartsWithEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StartsWithEvaluator.java
@@ -20,6 +20,7 @@ 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.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -34,13 +35,13 @@ public class StartsWithEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new BooleanQueryResult(false);
         }
 
-        final String searchString = search.evaluate(attributes).getValue();
+        final String searchString = search.evaluate(attributes, context).getValue();
         return new BooleanQueryResult(searchString == null ? false : subjectValue.startsWith(searchString));
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StringToDateEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StringToDateEvaluator.java
index 387777a..e7bc339 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StringToDateEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StringToDateEvaluator.java
@@ -25,6 +25,7 @@ import java.util.TimeZone;
 
 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.EvaluationContext;
 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.exception.IllegalAttributeException;
@@ -42,9 +43,9 @@ public class StringToDateEvaluator extends DateEvaluator {
     }
 
     @Override
-    public QueryResult<Date> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        final String formatValue = format.evaluate(attributes).getValue();
+    public QueryResult<Date> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
+        final String formatValue = format.evaluate(attributes, context).getValue();
         if (subjectValue == null || formatValue == null) {
             return new DateQueryResult(null);
         }
@@ -52,7 +53,7 @@ public class StringToDateEvaluator extends DateEvaluator {
         final SimpleDateFormat sdf = new SimpleDateFormat(formatValue, Locale.US);
 
         if(timeZone != null) {
-            final QueryResult<String> tzResult = timeZone.evaluate(attributes);
+            final QueryResult<String> tzResult = timeZone.evaluate(attributes, context);
             final String tz = tzResult.getValue();
             if(tz != null && TimeZone.getTimeZone(tz) != null) {
                 sdf.setTimeZone(TimeZone.getTimeZone(tz));
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterEvaluator.java
index 6625cb6..8370bb4 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -34,12 +35,12 @@ public class SubstringAfterEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new StringQueryResult("");
         }
-        final String afterValue = afterEvaluator.evaluate(attributes).getValue();
+        final String afterValue = afterEvaluator.evaluate(attributes, context).getValue();
         if (afterValue == null || afterValue.length() == 0) {
             return new StringQueryResult(subjectValue);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterLastEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterLastEvaluator.java
index 2cc9c3c..59b4f8e 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterLastEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterLastEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -34,12 +35,12 @@ public class SubstringAfterLastEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new StringQueryResult("");
         }
-        final String afterValue = afterEvaluator.evaluate(attributes).getValue();
+        final String afterValue = afterEvaluator.evaluate(attributes, context).getValue();
         final int index = subjectValue.lastIndexOf(afterValue);
         if (index < 0 || index >= subjectValue.length()) {
             return new StringQueryResult(subjectValue);
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeEvaluator.java
index 4b8c3d0..46b0a17 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -34,12 +35,12 @@ public class SubstringBeforeEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new StringQueryResult("");
         }
-        final String beforeValue = beforeEvaluator.evaluate(attributes).getValue();
+        final String beforeValue = beforeEvaluator.evaluate(attributes, context).getValue();
         if (beforeValue == null || beforeValue.length() == 0) {
             return new StringQueryResult(subjectValue);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeLastEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeLastEvaluator.java
index 14548c1..505e5b9 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeLastEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeLastEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -34,12 +35,12 @@ public class SubstringBeforeLastEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new StringQueryResult("");
         }
-        final String beforeValue = beforeEvaluator.evaluate(attributes).getValue();
+        final String beforeValue = beforeEvaluator.evaluate(attributes, context).getValue();
         final int index = subjectValue.lastIndexOf(beforeValue);
         if (index < 0) {
             return new StringQueryResult(subjectValue);
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringEvaluator.java
index 71b78f1..0297983 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -42,16 +43,16 @@ public class SubstringEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new StringQueryResult("");
         }
-        final int startIndexValue = startIndex.evaluate(attributes).getValue().intValue();
+        final int startIndexValue = startIndex.evaluate(attributes, context).getValue().intValue();
         if (endIndex == null) {
             return new StringQueryResult(subjectValue.substring(startIndexValue));
         } else {
-            final int endIndexValue = endIndex.evaluate(attributes).getValue().intValue();
+            final int endIndexValue = endIndex.evaluate(attributes, context).getValue().intValue();
             return new StringQueryResult(subjectValue.substring(startIndexValue, endIndexValue));
         }
     }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ThreadEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ThreadEvaluator.java
index 4ebfbe5..4c5a49e 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ThreadEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ThreadEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -26,7 +27,7 @@ import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResul
 public class ThreadEvaluator extends StringEvaluator {
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
         // See org.apache.nifi.engine.FlowEngine
         return new StringQueryResult(Thread.currentThread().getName());
     }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToLowerEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToLowerEvaluator.java
index 33175f8..4285390 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToLowerEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToLowerEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -32,8 +33,8 @@ public class ToLowerEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         return new StringQueryResult(subjectValue == null ? null : subjectValue.toLowerCase());
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToRadixEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToRadixEvaluator.java
index 1cf7e55..194ae0a 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToRadixEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToRadixEvaluator.java
@@ -19,6 +19,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 import java.util.Arrays;
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -41,20 +42,20 @@ public class ToRadixEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final Long result = numberEvaluator.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final Long result = numberEvaluator.evaluate(attributes, context).getValue();
         if (result == null) {
             return new StringQueryResult(null);
         }
 
-        final Long radix = radixEvaluator.evaluate(attributes).getValue();
+        final Long radix = radixEvaluator.evaluate(attributes, context).getValue();
         if (radix == null) {
             return new StringQueryResult(null);
         }
 
         String stringValue = Long.toString(result.longValue(), radix.intValue());
         if (minimumWidthEvaluator != null) {
-            final Long minimumWidth = minimumWidthEvaluator.evaluate(attributes).getValue();
+            final Long minimumWidth = minimumWidthEvaluator.evaluate(attributes, context).getValue();
             if (minimumWidth != null) {
                 final int paddingWidth = minimumWidth.intValue() - stringValue.length();
                 if (paddingWidth > 0) {
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToStringEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToStringEvaluator.java
index 2f7fbe3..36e1d7c 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToStringEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToStringEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -32,8 +33,8 @@ public class ToStringEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final Object result = subject.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final Object result = subject.evaluate(attributes, context).getValue();
         return new StringQueryResult(result == null ? null : result.toString());
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToUpperEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToUpperEvaluator.java
index f3174c2..a8269d1 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToUpperEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToUpperEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -32,8 +33,8 @@ public class ToUpperEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         return new StringQueryResult(subjectValue == null ? null : subjectValue.toUpperCase());
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/TrimEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/TrimEvaluator.java
index e44661f..3d6f988 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/TrimEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/TrimEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -32,8 +33,8 @@ public class TrimEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         return new StringQueryResult(subjectValue == null ? null : subjectValue.trim());
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlDecodeEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlDecodeEvaluator.java
index c61d55d..daa949e 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlDecodeEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlDecodeEvaluator.java
@@ -20,6 +20,7 @@ import java.io.UnsupportedEncodingException;
 import java.net.URLDecoder;
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -34,8 +35,8 @@ public class UrlDecodeEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new StringQueryResult(null);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlEncodeEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlEncodeEvaluator.java
index 769f6df..6b856ab 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlEncodeEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlEncodeEvaluator.java
@@ -20,6 +20,7 @@ import java.io.UnsupportedEncodingException;
 import java.net.URLEncoder;
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -34,8 +35,8 @@ public class UrlEncodeEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String subjectValue = subject.evaluate(attributes, context).getValue();
         if (subjectValue == null) {
             return new StringQueryResult(null);
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UuidEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UuidEvaluator.java
index faa8f0a..d95fe6e 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UuidEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UuidEvaluator.java
@@ -19,6 +19,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.functions;
 import java.util.Map;
 import java.util.UUID;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -27,7 +28,7 @@ import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResul
 public class UuidEvaluator extends StringEvaluator {
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
         return new StringQueryResult(UUID.randomUUID().toString());
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/BooleanLiteralEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/BooleanLiteralEvaluator.java
index a085423..0f179a4 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/BooleanLiteralEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/BooleanLiteralEvaluator.java
@@ -20,6 +20,7 @@ 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.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -32,7 +33,7 @@ public class BooleanLiteralEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
+    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
         return new BooleanQueryResult(value);
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/DecimalLiteralEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/DecimalLiteralEvaluator.java
index 9673e16..b8330ff 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/DecimalLiteralEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/DecimalLiteralEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.literals;
 
 import org.apache.nifi.attribute.expression.language.evaluation.DecimalEvaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.DecimalQueryResult;
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -33,7 +34,7 @@ public class DecimalLiteralEvaluator extends DecimalEvaluator {
     }
 
     @Override
-    public QueryResult<Double> evaluate(final Map<String, String> attributes) {
+    public QueryResult<Double> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
         return new DecimalQueryResult(literal);
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/StringLiteralEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/StringLiteralEvaluator.java
index d739ac7..c372f04 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/StringLiteralEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/StringLiteralEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.literals;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -65,7 +66,7 @@ public class StringLiteralEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
         return new StringQueryResult(value);
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/ToLiteralEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/ToLiteralEvaluator.java
index ea683ad..240abc0 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/ToLiteralEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/ToLiteralEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.literals;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -31,8 +32,8 @@ public class ToLiteralEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final Object result = argEvaluator.evaluate(attributes);
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final Object result = argEvaluator.evaluate(attributes, context);
         return new StringQueryResult(result == null ? null : result.toString());
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/WholeNumberLiteralEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/WholeNumberLiteralEvaluator.java
index 7ca6255..5155b6c 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/WholeNumberLiteralEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/WholeNumberLiteralEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.literals;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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.WholeNumberEvaluator;
@@ -32,7 +33,7 @@ public class WholeNumberLiteralEvaluator extends WholeNumberEvaluator {
     }
 
     @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
+    public QueryResult<Long> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
         return new WholeNumberQueryResult(literal);
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/CountEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/CountEvaluator.java
index 5026846..d443152 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/CountEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/CountEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.reduce;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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.WholeNumberEvaluator;
@@ -27,15 +28,18 @@ import org.apache.nifi.expression.AttributeExpression.ResultType;
 public class CountEvaluator extends WholeNumberEvaluator implements ReduceEvaluator<Long> {
 
     private final Evaluator<?> subjectEvaluator;
-    private long count = 0L;
 
     public CountEvaluator(final Evaluator<?> subjectEvaluator) {
         this.subjectEvaluator = subjectEvaluator;
     }
 
     @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
-        final QueryResult<?> result = subjectEvaluator.evaluate(attributes);
+    public QueryResult<Long> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final QueryResult<?> result = subjectEvaluator.evaluate(attributes, context);
+        Long count = context.getState(this, Long.class);
+        if (count == null) {
+            count = 0L;
+        }
         if (result.getValue() == null) {
             return new WholeNumberQueryResult(count);
         }
@@ -45,6 +49,7 @@ public class CountEvaluator extends WholeNumberEvaluator implements ReduceEvalua
         }
 
         count++;
+        context.putState(this, count);
         return new WholeNumberQueryResult(count);
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/JoinEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/JoinEvaluator.java
index f80924b..40efaa8 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/JoinEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/JoinEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.reduce;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -28,33 +29,40 @@ public class JoinEvaluator extends StringEvaluator implements ReduceEvaluator<St
     private final Evaluator<String> subjectEvaluator;
     private final Evaluator<String> delimiterEvaluator;
 
-    private final StringBuilder sb = new StringBuilder();
-    private int evalCount = 0;
-
     public JoinEvaluator(final Evaluator<String> subject, final Evaluator<String> delimiter) {
         this.subjectEvaluator = subject;
         this.delimiterEvaluator = delimiter;
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        String subject = subjectEvaluator.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        String subject = subjectEvaluator.evaluate(attributes, context).getValue();
         if (subject == null) {
             subject = "";
         }
 
-        final String delimiter = delimiterEvaluator.evaluate(attributes).getValue();
-        if (evalCount > 0) {
-            sb.append(delimiter);
+        final String delimiter = delimiterEvaluator.evaluate(attributes, context).getValue();
+        State state = context.getState(this, State.class);
+        if (state == null) {
+            state = new State();
+            context.putState(this, state);
+        }
+        if (state.evalCount > 0) {
+            state.sb.append(delimiter);
         }
-        sb.append(subject);
+        state.sb.append(subject);
 
-        evalCount++;
-        return new StringQueryResult(sb.toString());
+        state.evalCount++;
+        return new StringQueryResult(state.sb.toString());
     }
 
     @Override
     public Evaluator<?> getSubjectEvaluator() {
         return subjectEvaluator;
     }
+
+    private class State {
+        private final StringBuilder sb = new StringBuilder();
+        private int evalCount = 0;
+    }
 }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java
index 8c4213b..643995c 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java
@@ -20,6 +20,7 @@ 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.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -34,8 +35,8 @@ public class AllAttributesEvaluator extends BooleanEvaluator implements Iteratin
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        QueryResult<Boolean> attributeValueQuery = booleanEvaluator.evaluate(attributes);
+    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        QueryResult<Boolean> attributeValueQuery = booleanEvaluator.evaluate(attributes, context);
         Boolean result = attributeValueQuery.getValue();
         if (result == null) {
             return new BooleanQueryResult(false);
@@ -45,8 +46,8 @@ public class AllAttributesEvaluator extends BooleanEvaluator implements Iteratin
             return new BooleanQueryResult(false);
         }
 
-        while (multiAttributeEvaluator.getEvaluationsRemaining() > 0) {
-            attributeValueQuery = booleanEvaluator.evaluate(attributes);
+        while (multiAttributeEvaluator.getEvaluationsRemaining(context) > 0) {
+            attributeValueQuery = booleanEvaluator.evaluate(attributes, context);
             result = attributeValueQuery.getValue();
             if (result != null && !result) {
                 return attributeValueQuery;
@@ -57,7 +58,7 @@ public class AllAttributesEvaluator extends BooleanEvaluator implements Iteratin
     }
 
     @Override
-    public int getEvaluationsRemaining() {
+    public int getEvaluationsRemaining(final EvaluationContext context) {
         return 0;
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java
index eac571b..b8e75c1 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java
@@ -20,6 +20,7 @@ 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.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -34,8 +35,8 @@ public class AnyAttributeEvaluator extends BooleanEvaluator implements Iterating
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        QueryResult<Boolean> attributeValueQuery = booleanEvaluator.evaluate(attributes);
+    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        QueryResult<Boolean> attributeValueQuery = booleanEvaluator.evaluate(attributes, context);
         Boolean result = attributeValueQuery.getValue();
         if (result == null) {
             return new BooleanQueryResult(false);
@@ -45,8 +46,8 @@ public class AnyAttributeEvaluator extends BooleanEvaluator implements Iterating
             return new BooleanQueryResult(true);
         }
 
-        while (multiAttributeEvaluator.getEvaluationsRemaining() > 0) {
-            attributeValueQuery = booleanEvaluator.evaluate(attributes);
+        while (multiAttributeEvaluator.getEvaluationsRemaining(context) > 0) {
+            attributeValueQuery = booleanEvaluator.evaluate(attributes, context);
             result = attributeValueQuery.getValue();
             if (result != null && result) {
                 return attributeValueQuery;
@@ -57,7 +58,7 @@ public class AnyAttributeEvaluator extends BooleanEvaluator implements Iterating
     }
 
     @Override
-    public int getEvaluationsRemaining() {
+    public int getEvaluationsRemaining(final EvaluationContext context) {
         return 0;
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AttributeEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AttributeEvaluator.java
index 0da063e..76150b1 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AttributeEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AttributeEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.selection;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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;
@@ -32,8 +33,8 @@ public class AttributeEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String nameValue = nameEvaluator.evaluate(attributes).getValue();
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        final String nameValue = nameEvaluator.evaluate(attributes, context).getValue();
         final String attributeValue = attributes.get(nameValue);
         return new StringQueryResult(attributeValue);
     }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/DelineatedAttributeEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/DelineatedAttributeEvaluator.java
index 44a3d55..8df0202 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/DelineatedAttributeEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/DelineatedAttributeEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.selection;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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.StringQueryResult;
@@ -27,9 +28,6 @@ public class DelineatedAttributeEvaluator extends MultiAttributeEvaluator {
     private final Evaluator<String> subjectEvaluator;
     private final Evaluator<String> delimiterEvaluator;
     private final int evaluationType;
-    private String[] delineatedValues;
-    private int evaluationCount = 0;
-    private int evaluationsLeft = 1;
 
     public DelineatedAttributeEvaluator(final Evaluator<String> subjectEvaluator, final Evaluator<String> delimiterEvaluator, final int evaluationType) {
         this.subjectEvaluator = subjectEvaluator;
@@ -38,31 +36,36 @@ public class DelineatedAttributeEvaluator extends MultiAttributeEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        if (delineatedValues == null) {
-            final QueryResult<String> subjectValue = subjectEvaluator.evaluate(attributes);
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        State state = context.getState(this, State.class);
+        if (state == null) {
+            state = new State();
+            context.putState(this, state);
+        }
+        if (state.delineatedValues == null) {
+            final QueryResult<String> subjectValue = subjectEvaluator.evaluate(attributes, context);
             if (subjectValue.getValue() == null) {
-                evaluationsLeft = 0;
+                state.evaluationsLeft = 0;
                 return new StringQueryResult(null);
             }
 
-            final QueryResult<String> delimiterValue = delimiterEvaluator.evaluate(attributes);
-            if (subjectValue.getValue() == null) {
-                evaluationsLeft = 0;
+            final QueryResult<String> delimiterValue = delimiterEvaluator.evaluate(attributes, context);
+            if (delimiterValue.getValue() == null) {
+                state.evaluationsLeft = 0;
                 return new StringQueryResult(null);
             }
 
-            delineatedValues = subjectValue.getValue().split(delimiterValue.getValue());
+            state.delineatedValues = subjectValue.getValue().split(delimiterValue.getValue());
         }
 
-        if (evaluationCount > delineatedValues.length || delineatedValues.length == 0) {
-            evaluationsLeft = 0;
+        if (state.evaluationCount > state.delineatedValues.length || state.delineatedValues.length == 0) {
+            state.evaluationsLeft = 0;
             return new StringQueryResult(null);
         }
 
-        evaluationsLeft = delineatedValues.length - evaluationCount - 1;
+        state.evaluationsLeft = state.delineatedValues.length - state.evaluationCount - 1;
 
-        return new StringQueryResult(delineatedValues[evaluationCount++]);
+        return new StringQueryResult(state.delineatedValues[state.evaluationCount++]);
     }
 
     @Override
@@ -71,8 +74,13 @@ public class DelineatedAttributeEvaluator extends MultiAttributeEvaluator {
     }
 
     @Override
-    public int getEvaluationsRemaining() {
-        return evaluationsLeft;
+    public int getEvaluationsRemaining(final EvaluationContext context) {
+        State state = context.getState(this, State.class);
+        if (state == null) {
+            state = new State();
+            context.putState(this, state);
+        }
+        return state.evaluationsLeft;
     }
 
     @Override
@@ -84,4 +92,10 @@ public class DelineatedAttributeEvaluator extends MultiAttributeEvaluator {
     public int getEvaluationType() {
         return evaluationType;
     }
+
+    private class State {
+        private String[] delineatedValues;
+        private int evaluationCount = 0;
+        private int evaluationsLeft = 1;
+    }
 }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MappingEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MappingEvaluator.java
index 5f182c7..a8a79ea 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MappingEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MappingEvaluator.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language.evaluation.selection;
 
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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.reduce.ReduceEvaluator;
@@ -34,11 +35,11 @@ public class MappingEvaluator<T> implements Evaluator<T> {
     }
 
     @Override
-    public QueryResult<T> evaluate(final Map<String, String> attributes) {
-        QueryResult<T> result = mappingEvaluator.evaluate(attributes);
+    public QueryResult<T> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        QueryResult<T> result = mappingEvaluator.evaluate(attributes, context);
 
-        while (multiAttributeEvaluator.getEvaluationsRemaining() > 0) {
-            result = mappingEvaluator.evaluate(attributes);
+        while (multiAttributeEvaluator.getEvaluationsRemaining(context) > 0) {
+            result = mappingEvaluator.evaluate(attributes, context);
         }
 
         return result;
@@ -50,7 +51,7 @@ public class MappingEvaluator<T> implements Evaluator<T> {
     }
 
     @Override
-    public int getEvaluationsRemaining() {
+    public int getEvaluationsRemaining(final EvaluationContext context) {
         return 0;
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiMatchAttributeEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiMatchAttributeEvaluator.java
index 8292f84..0f37c9f 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiMatchAttributeEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiMatchAttributeEvaluator.java
@@ -21,6 +21,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.regex.Pattern;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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.StringQueryResult;
@@ -29,8 +30,6 @@ public class MultiMatchAttributeEvaluator extends MultiAttributeEvaluator {
 
     private final List<Pattern> attributePatterns;
     private final int evaluationType;
-    private final List<String> attributeNames = new ArrayList<>();
-    private int evaluationCount = 0;
 
     public MultiMatchAttributeEvaluator(final List<String> attributeRegexes, final int evaluationType) {
         this.attributePatterns = new ArrayList<>();
@@ -47,27 +46,37 @@ public class MultiMatchAttributeEvaluator extends MultiAttributeEvaluator {
      * @return number of remaining evaluations
      */
     @Override
-    public int getEvaluationsRemaining() {
-        return attributeNames.size() - evaluationCount;
+    public int getEvaluationsRemaining(final EvaluationContext context) {
+        State state = context.getState(this, State.class);
+        if (state == null) {
+            state = new State();
+            context.putState(this, state);
+        }
+        return state.attributeNames.size() - state.evaluationCount;
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        if (evaluationCount == 0) {
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        State state = context.getState(this, State.class);
+        if (state == null) {
+            state = new State();
+            context.putState(this, state);
+        }
+        if (state.evaluationCount == 0) {
             for (final Pattern pattern : attributePatterns) {
                 for (final String attrName : attributes.keySet()) {
                     if (pattern.matcher(attrName).matches()) {
-                        attributeNames.add(attrName);
+                        state.attributeNames.add(attrName);
                     }
                 }
             }
         }
 
-        if (evaluationCount >= attributeNames.size()) {
+        if (state.evaluationCount >= state.attributeNames.size()) {
             return new StringQueryResult(null);
         }
 
-        return new StringQueryResult(attributes.get(attributeNames.get(evaluationCount++)));
+        return new StringQueryResult(attributes.get(state.attributeNames.get(state.evaluationCount++)));
     }
 
     @Override
@@ -84,4 +93,9 @@ public class MultiMatchAttributeEvaluator extends MultiAttributeEvaluator {
     public Evaluator<?> getLogicEvaluator() {
         return this;
     }
+
+    private class State {
+        private final List<String> attributeNames = new ArrayList<>();
+        private int evaluationCount = 0;
+    }
 }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java
index cff9185..0f14dcc 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java
@@ -20,6 +20,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluationContext;
 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.StringQueryResult;
@@ -28,8 +29,6 @@ public class MultiNamedAttributeEvaluator extends MultiAttributeEvaluator {
 
     private final List<String> attributeNames;
     private final int evaluationType;
-    private int evaluationCount = 0;
-    private List<String> matchingAttributeNames = null;
 
     public MultiNamedAttributeEvaluator(final List<String> attributeNames, final int evaluationType) {
         this.attributeNames = attributeNames;
@@ -37,19 +36,29 @@ public class MultiNamedAttributeEvaluator extends MultiAttributeEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        matchingAttributeNames = new ArrayList<>(attributeNames);
+    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluationContext context) {
+        State state = context.getState(this, State.class);
+        if (state == null) {
+            state = new State();
+            context.putState(this, state);
+        }
+        state.matchingAttributeNames = new ArrayList<>(attributeNames);
 
-        if (matchingAttributeNames.size() <= evaluationCount) {
+        if (state.matchingAttributeNames.size() <= state.evaluationCount) {
             return new StringQueryResult(null);
         }
 
-        return new StringQueryResult(attributes.get(matchingAttributeNames.get(evaluationCount++)));
+        return new StringQueryResult(attributes.get(state.matchingAttributeNames.get(state.evaluationCount++)));
     }
 
     @Override
-    public int getEvaluationsRemaining() {
-        return matchingAttributeNames.size() - evaluationCount;
+    public int getEvaluationsRemaining(final EvaluationContext context) {
+        State state = context.getState(this, State.class);
+        if (state == null) {
+            state = new State();
+            context.putState(this, state);
+        }
+        return state.matchingAttributeNames.size() - state.evaluationCount;
     }
 
     @Override
@@ -70,4 +79,9 @@ public class MultiNamedAttributeEvaluator extends MultiAttributeEvaluator {
     public List<String> getAttributeNames() {
         return attributeNames;
     }
+
+    private class State {
+        private int evaluationCount = 0;
+        private List<String> matchingAttributeNames = null;
+    }
 }
diff --git a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java
index ea7cb96..556066f 100644
--- a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java
+++ b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java
@@ -90,6 +90,152 @@ public class TestStandardPreparedQuery {
     }
 
     @Test
+    public void testPreparedQueryWithReducingFunctionAny() {
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("comma", ",");
+        attributes.put("question", " how are you?");
+        final StandardPreparedQuery prepared = (StandardPreparedQuery) Query.prepare("${anyAttribute('comma', 'question'):matches('hello')}");
+        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+        attributes.put("audience", "bla");
+        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+        attributes.put("comma", "hello");
+        assertEquals("true", prepared.evaluateExpressions(attributes, null));
+    }
+
+    @Test
+    public void testPreparedQueryWithReducingFunctionAll() {
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("comma", ",");
+        attributes.put("question", " how are you?");
+        final StandardPreparedQuery prepared = (StandardPreparedQuery) Query.prepare("${allAttributes('comma', 'question'):matches('hello')}");
+        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+        attributes.put("comma", "hello");
+        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+        attributes.put("question", "hello");
+        assertEquals("true", prepared.evaluateExpressions(attributes, null));
+    }
+
+    @Test
+    public void testPreparedQueryWithReducingFunctionAnyMatching() {
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("comma", ",");
+        attributes.put("question", " how are you?");
+        final StandardPreparedQuery prepared = (StandardPreparedQuery) Query.prepare("${anyMatchingAttribute('audi.*'):matches('hello')}");
+        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+        attributes.put("audience", "bla");
+        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+        attributes.put("auditorium", "hello");
+        assertEquals("true", prepared.evaluateExpressions(attributes, null));
+    }
+
+    @Test
+    public void testPreparedQueryWithReducingFunctionAllMatching() {
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("comma", "hello");
+        attributes.put("question", "hello");
+        final StandardPreparedQuery prepared = (StandardPreparedQuery) Query.prepare("${allMatchingAttributes('.*'):matches('hello')}");
+        assertEquals("true", prepared.evaluateExpressions(attributes, null));
+        assertEquals("true", prepared.evaluateExpressions(attributes, null));
+        attributes.put("audience", "bla");
+        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+        attributes.remove("audience");
+        assertEquals("true", prepared.evaluateExpressions(attributes, null));
+    }
+
+    @Test
+    public void testPreparedQueryWithReducingFunctionAnyDelineated() {
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("number_list", "1,2,3,4,5,6,7");
+        final StandardPreparedQuery prepared = (StandardPreparedQuery) Query.prepare("${anyDelineatedValue(${number_list}, ','):contains('5')}");
+        assertEquals("true", prepared.evaluateExpressions(attributes, null));
+        assertEquals("true", prepared.evaluateExpressions(attributes, null));
+        attributes.put("number_list", "1,2,3");
+        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+        attributes.put("number_list", "5");
+        assertEquals("true", prepared.evaluateExpressions(attributes, null));
+    }
+
+    @Test
+    public void testPreparedQueryWithReducingFunctionAllDelineated() {
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("word_list", "beach,bananas,web");
+        final StandardPreparedQuery prepared = (StandardPreparedQuery) Query.prepare("${allDelineatedValues(${word_list}, ','):contains('b')}");
+        assertEquals("true", prepared.evaluateExpressions(attributes, null));
+        assertEquals("true", prepared.evaluateExpressions(attributes, null));
+        attributes.put("word_list", "beach,party,web");
+        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+        attributes.put("word_list", "bee");
+        assertEquals("true", prepared.evaluateExpressions(attributes, null));
+    }
+
+    @Test
+    public void testPreparedQueryWithReducingFunctionJoin() {
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("hello", "Hello");
+        attributes.put("boat", "World!");
+        final StandardPreparedQuery prepared = (StandardPreparedQuery) Query.prepare("${allAttributes('hello', 'boat'):join(' ')}");
+        assertEquals("Hello World!", prepared.evaluateExpressions(attributes, null));
+        attributes.put("boat", "Friend.");
+        assertEquals("Hello Friend.", prepared.evaluateExpressions(attributes, null));
+    }
+
+    @Test
+    public void testPreparedQueryWithReducingFunctionCount() {
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("hello", "Hello");
+        attributes.put("boat", "World!");
+        final StandardPreparedQuery prepared = (StandardPreparedQuery) Query.prepare("${allAttributes('hello', 'boat'):contains('e'):count()}");
+        assertEquals("1", prepared.evaluateExpressions(attributes, null));
+        attributes.put("boat", "end");
+        assertEquals("2", prepared.evaluateExpressions(attributes, null));
+    }
+
+    @Test
+    public void testPreparedQueryWithSelectingAnd() {
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("hello", "Hello");
+        attributes.put("boat", "World!");
+        final StandardPreparedQuery prepared = (StandardPreparedQuery) Query.prepare("${allAttributes('hello', 'boat'):isEmpty():not():and(${hello:contains('o')})}");
+        assertEquals("true", prepared.evaluateExpressions(attributes, null));
+        attributes.put("hello", "hi");
+        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+    }
+
+    @Test
+    public void testPreparedQueryWithAnd() {
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("hello", "Hello");
+        final StandardPreparedQuery prepared = (StandardPreparedQuery) Query.prepare("${hello:contains('H'):and(${hello:contains('o')})}");
+        assertEquals("true", prepared.evaluateExpressions(attributes, null));
+        attributes.put("hello", "Hell");
+        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+    }
+
+    @Test
+    public void testPreparedQueryWithSelectingOr() {
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("hello", "Hello");
+        attributes.put("boat", "World!");
+        final StandardPreparedQuery prepared = (StandardPreparedQuery) Query.prepare("${allAttributes('hello', 'boat'):matches('strict'):or(${hello:contains('o')})}");
+        assertEquals("true", prepared.evaluateExpressions(attributes, null));
+        attributes.put("hello", "hi");
+        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+    }
+
+    @Test
+    public void testPreparedQueryWithOr() {
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("hello", "Hello");
+        final StandardPreparedQuery prepared = (StandardPreparedQuery) Query.prepare("${hello:contains('X'):or(${hello:contains('o')})}");
+        assertEquals("true", prepared.evaluateExpressions(attributes, null));
+        attributes.put("hello", "Hell");
+        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+    }
+
+    @Test
     public void testVariableImpacted() {
         final Set<String> attr = new HashSet<>();
         attr.add("attr");