You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by mc...@apache.org on 2019/08/02 14:48:25 UTC

[nifi] branch master updated: NIFI-6380: Introduced the notion of Parameters and Parameter Contexts to the code base. - Added nifi-parameter module - Added Parameter to nifi-api - Added ParameterContext and ParameterLookup - Updated EL to support Parameters - Updated backend so that any property can make use of Parameters - Added web endpoint for Parameter Contexts - Updated Templates to make use of Parameter Contexts - Updated Versioned Flows to make use of Parameter Contexts - Updated Stateless NiFi to support Parame [...]

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 4f50f30  NIFI-6380: Introduced the notion of Parameters and Parameter Contexts to the code base. - Added nifi-parameter module - Added Parameter to nifi-api - Added ParameterContext and ParameterLookup - Updated EL to support Parameters - Updated backend so that any property can make use of Parameters - Added web endpoint for Parameter Contexts - Updated Templates to make use of Parameter Contexts - Updated Versioned Flows to make use of Parameter Contexts - Updated Stateless NiF [...]
4f50f30 is described below

commit 4f50f30ad7329540a707be835c23acee86d4cfb4
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Thu May 2 11:33:10 2019 -0400

    NIFI-6380: Introduced the notion of Parameters and Parameter Contexts to the code base.
    - Added nifi-parameter module
    - Added Parameter to nifi-api
    - Added ParameterContext and ParameterLookup
    - Updated EL to support Parameters
    - Updated backend so that any property can make use of Parameters
    - Added web endpoint for Parameter Contexts
    - Updated Templates to make use of Parameter Contexts
    - Updated Versioned Flows to make use of Parameter Contexts
    - Updated Stateless NiFi to support Parameters and take Parameters as part of the configuration, not Variables.
    
    NIFI-6380: Addressed review feedback and fixed bugs; added additional unit and integration tests to verify
    
    NIFI-6380: Added Description to Parameter Context
    
    NIFI-6380: Fixed checkstyle violations
    
    NIFI-6380: Fixed bug that caused updating a Parameter Context to fail if the name is provided and unchanged
    
    NIFI-6380: If parameter is being deleted, don't worry about its sensitivity flag. This addresses a bug where the deletion of a Sensitive Parameter would require that the Parameter be submitted with a value of null and a sensitivity flag of true; else it would provide an error indicating that the parameter can't be changed from sensitive to non-sensitive. Now, the sensitivity flag is ignored.
    
    NIFI-6380: Fixed bug around unsetting Process Group's Parameter Context
    
    NIFI-6380: Moved lastRefreshed timestamp from ParameterContextDTO to ParameterContextsEntity and renamed to currentTime to match the pattern of ControllerServicesEntity rather than FlowHistory. Added parameterContextId to ProcessGroupFlowDTO
    
    NIFI-6380: Added additional integration tests around escaped parameter references
    
    NIFI-6380: Additional tests and bug fix for referencing EL from within another EL Expression
    
    NIFI-6380: Created ParameterEntity to house a 'canWrite' flag for parameters and updated ParameterContext to use it. Updated ParameterContextUpdateRequestDTO to include a Set<AffectedComponentEntity> indicating the components that are affected by the update
    
    NIFI-6380: Addressed review feedback
    
    NIFI-6380: Addressed additional review feedback, mostly around code cleanup
    
    NIFI-6380: Bug fix
    
    NIFI-6380: Addressed more review feedback; fixed a couple of minor bugs encountered when testing
    
    NIFI-6380: Bug fix around Parameter escaping for properties that support Expression Language. Fixed inconcsistency in Authorization hierarchy between /parameter-contexts/1234 and /parameter-contexts  to ensure that the parent of /parameter-contexts is /controller, regardless of how we arrive at the /parameter-contexts resource
    
    NIFI-6380: Fixed but around using ProcessContext#newPropertyValue(String) that previously resulted in causing the Parameters to be evaluated a second time, thereby ignoring escape characters. Updated Integration Test to verify behavior. Also fixed bug in LoadBalanceQueueIT as it was a one-liner and noticed it while running tests to verify other behavior
    
    NIFI-6380: Ensure that if Processor or Controller Services references or de-references a Parameter that user making the change has READ policy on the Parameter Context
    
    NIFI-6380: Verify permissions on Parameter Context when creating processor/controller service and when moving between groups, insantiating template, reverting/change flow version
    
    NIFI-6380: Addressed problem that resulted from rebasing against master; fixed bug in showing the Affected Components for a particular Parameter Context Update
    
    NIFI-6380: Fixed NPE that occurs when attempting to update controller-level Controller Service
    
    NIFI-6380: Updated validation error text when referencing parameter from controller-level controller service or reporting task
    
    NIFI-6380: Ensuring permissions for copy/paste when components reference parameters.
    
    NIFI-6380: Do not require READ policy on Parameter Context in order to un-reference a parameter, only to reference one.
    
    NIFI-6380: Restricted parameter names to match those in the Feature Proposal. Fixed bug that allowed copy & paste of a PG for which user does not have READ policy for a child/descendant PG's parameter context
    
    This closes #3536
---
 .../components/AbstractConfigurableComponent.java  |    2 +
 .../apache/nifi/components/ValidationContext.java  |   17 +-
 .../java/org/apache/nifi/parameter/Parameter.java  |   40 +-
 .../apache/nifi/parameter/ParameterDescriptor.java |  115 ++
 .../nifi/bootstrap/NotificationServiceManager.java |   46 +-
 .../NotificationValidationContext.java             |   27 +-
 nifi-commons/nifi-expression-language/pom.xml      |    7 +-
 .../language/antlr/AttributeExpressionLexer.g      |    5 +-
 .../language/antlr/AttributeExpressionParser.g     |   16 +-
 .../expression/language/CompiledExpression.java    |    9 +-
 .../expression/language/EmptyPreparedQuery.java    |    9 +-
 ...iteralEvaluator.java => EvaluationContext.java} |   38 +-
 .../attribute/expression/language/Expression.java  |    7 +-
 .../expression/language/InvalidPreparedQuery.java  |    9 +-
 ...ralExpression.java => ParameterExpression.java} |   17 +-
 .../expression/language/PreparedQuery.java         |    6 +-
 .../nifi/attribute/expression/language/Query.java  |  160 ++-
 .../language/StandardAttributeExpression.java      |   15 +-
 .../language/StandardEvaluationContext.java        |   71 ++
 .../StandardExpressionLanguageCompiler.java        |   13 +-
 .../expression/language/StandardPreparedQuery.java |   15 +-
 .../expression/language/StandardPropertyValue.java |   28 +-
 .../language/StringLiteralExpression.java          |    4 +-
 .../attribute/expression/language/ValueLookup.java |   20 +-
 .../language/compile/ExpressionCompiler.java       |   56 +-
 .../language/evaluation/BooleanEvaluator.java      |    3 +-
 .../language/evaluation/DateEvaluator.java         |    7 +-
 .../language/evaluation/DateQueryResult.java       |    4 +-
 .../language/evaluation/DecimalEvaluator.java      |    3 +-
 .../expression/language/evaluation/Evaluator.java  |    7 +-
 .../language/evaluation/NumberEvaluator.java       |    3 +-
 .../language/evaluation/StringEvaluator.java       |    3 +-
 .../language/evaluation/WholeNumberEvaluator.java  |    3 +-
 .../evaluation/cast/BooleanCastEvaluator.java      |    8 +-
 .../evaluation/cast/DateCastEvaluator.java         |   21 +-
 .../evaluation/cast/DecimalCastEvaluator.java      |    8 +-
 .../evaluation/cast/NumberCastEvaluator.java       |    8 +-
 .../evaluation/cast/StringCastEvaluator.java       |    8 +-
 .../evaluation/cast/WholeNumberCastEvaluator.java  |    8 +-
 .../evaluation/functions/AndEvaluator.java         |   14 +-
 .../evaluation/functions/AppendEvaluator.java      |   10 +-
 .../functions/Base64DecodeEvaluator.java           |   13 +-
 .../functions/Base64EncodeEvaluator.java           |   19 +-
 .../functions/CharSequenceTranslatorEvaluator.java |    8 +-
 .../evaluation/functions/ContainsEvaluator.java    |   10 +-
 .../evaluation/functions/DivideEvaluator.java      |   10 +-
 .../evaluation/functions/EndsWithEvaluator.java    |   10 +-
 .../evaluation/functions/EqualsEvaluator.java      |   13 +-
 .../functions/EqualsIgnoreCaseEvaluator.java       |   10 +-
 .../evaluation/functions/FindEvaluator.java        |   17 +-
 .../evaluation/functions/FormatEvaluator.java      |   21 +-
 .../evaluation/functions/FromRadixEvaluator.java   |   10 +-
 .../functions/GetDelimitedFieldEvaluator.java      |   18 +-
 .../functions/GetStateVariableEvaluator.java       |   19 +-
 .../evaluation/functions/GreaterThanEvaluator.java |   10 +-
 .../functions/GreaterThanOrEqualEvaluator.java     |   10 +-
 .../evaluation/functions/HostnameEvaluator.java    |   12 +-
 .../language/evaluation/functions/IPEvaluator.java |   11 +-
 .../evaluation/functions/IfElseEvaluator.java      |   10 +-
 .../language/evaluation/functions/InEvaluator.java |   13 +-
 .../evaluation/functions/IndexOfEvaluator.java     |   11 +-
 .../evaluation/functions/IsEmptyEvaluator.java     |    8 +-
 .../evaluation/functions/IsNullEvaluator.java      |    8 +-
 .../evaluation/functions/JsonPathEvaluator.java    |   31 +-
 .../evaluation/functions/LastIndexOfEvaluator.java |   10 +-
 .../evaluation/functions/LengthEvaluator.java      |    8 +-
 .../evaluation/functions/LessThanEvaluator.java    |   10 +-
 .../functions/LessThanOrEqualEvaluator.java        |   10 +-
 .../evaluation/functions/MatchesEvaluator.java     |   17 +-
 .../evaluation/functions/MathEvaluator.java        |   17 +-
 .../evaluation/functions/MinusEvaluator.java       |   10 +-
 .../evaluation/functions/ModEvaluator.java         |   10 +-
 .../evaluation/functions/MultiplyEvaluator.java    |   10 +-
 .../evaluation/functions/NotEvaluator.java         |    8 +-
 .../evaluation/functions/NotNullEvaluator.java     |    8 +-
 .../evaluation/functions/NowEvaluator.java         |    9 +-
 .../functions/NumberToDateEvaluator.java           |   11 +-
 .../functions/OneUpSequenceEvaluator.java          |    9 +-
 .../language/evaluation/functions/OrEvaluator.java |   14 +-
 .../evaluation/functions/PlusEvaluator.java        |   10 +-
 .../evaluation/functions/PrependEvaluator.java     |   10 +-
 .../functions/RandomNumberGeneratorEvaluator.java  |    9 +-
 .../evaluation/functions/ReplaceAllEvaluator.java  |   12 +-
 .../functions/ReplaceEmptyEvaluator.java           |   10 +-
 .../evaluation/functions/ReplaceEvaluator.java     |   12 +-
 .../functions/ReplaceFirstEvaluator.java           |   12 +-
 .../evaluation/functions/ReplaceNullEvaluator.java |   10 +-
 .../evaluation/functions/StartsWithEvaluator.java  |   10 +-
 .../functions/StringToDateEvaluator.java           |   23 +-
 .../functions/SubstringAfterEvaluator.java         |   10 +-
 .../functions/SubstringAfterLastEvaluator.java     |   10 +-
 .../functions/SubstringBeforeEvaluator.java        |   10 +-
 .../functions/SubstringBeforeLastEvaluator.java    |   10 +-
 .../evaluation/functions/SubstringEvaluator.java   |   12 +-
 .../evaluation/functions/ThreadEvaluator.java      |    6 +-
 .../evaluation/functions/ToLowerEvaluator.java     |    8 +-
 .../evaluation/functions/ToRadixEvaluator.java     |   15 +-
 .../evaluation/functions/ToStringEvaluator.java    |    8 +-
 .../evaluation/functions/ToUpperEvaluator.java     |    8 +-
 .../evaluation/functions/TrimEvaluator.java        |    8 +-
 .../evaluation/functions/UrlDecodeEvaluator.java   |   13 +-
 .../evaluation/functions/UrlEncodeEvaluator.java   |   13 +-
 .../evaluation/functions/UuidEvaluator.java        |    9 +-
 .../literals/BooleanLiteralEvaluator.java          |    6 +-
 .../literals/DecimalLiteralEvaluator.java          |    6 +-
 .../literals/StringLiteralEvaluator.java           |    6 +-
 .../evaluation/literals/ToLiteralEvaluator.java    |    8 +-
 .../literals/WholeNumberLiteralEvaluator.java      |    6 +-
 .../language/evaluation/reduce/CountEvaluator.java |   12 +-
 .../language/evaluation/reduce/JoinEvaluator.java  |   14 +-
 .../selection/AllAttributesEvaluator.java          |   14 +-
 .../selection/AnyAttributeEvaluator.java           |   14 +-
 .../evaluation/selection/AttributeEvaluator.java   |   10 +-
 .../selection/DelineatedAttributeEvaluator.java    |   22 +-
 .../evaluation/selection/MappingEvaluator.java     |   14 +-
 .../selection/MultiMatchAttributeEvaluator.java    |   28 +-
 .../selection/MultiNamedAttributeEvaluator.java    |   25 +-
 .../ParameterEvaluator.java}                       |   24 +-
 .../expression/language/QueryGroovyTest.groovy     |   14 +-
 .../attribute/expression/language/TestQuery.java   |   80 +-
 .../language/TestStandardPreparedQuery.java        |   89 +-
 .../expression/language/TestValueLookup.java       |   10 +-
 nifi-commons/nifi-parameter/pom.xml                |   37 +
 .../nifi/parameter/AbstractParameterParser.java    |   56 +
 .../nifi/parameter/EscapedParameterReference.java} |   41 +-
 .../ExpressionLanguageAgnosticParameterParser.java |   71 ++
 .../ExpressionLanguageAwareParameterParser.java    |  110 ++
 .../apache/nifi/parameter/ParameterLookup.java}    |   42 +-
 .../apache/nifi/parameter/ParameterParser.java}    |   26 +-
 .../apache/nifi/parameter/ParameterReference.java} |   23 +-
 .../org/apache/nifi/parameter/ParameterToken.java  |   59 +
 .../apache/nifi/parameter/ParameterTokenList.java} |   19 +-
 .../nifi/parameter/StandardParameterReference.java |   73 ++
 .../nifi/parameter/StandardParameterTokenList.java |  105 ++
 .../nifi/parameter/StartCharacterEscape.java}      |   36 +-
 ...tExpressionLanguageAgnosticParameterParser.java |  190 +++
 ...TestExpressionLanguageAwareParameterParser.java |  201 ++++
 .../parameter/TestStandardParameterTokenList.java  |  106 ++
 nifi-commons/pom.xml                               |    3 +-
 nifi-mock/pom.xml                                  |    5 +
 .../org/apache/nifi/util/MockPropertyValue.java    |   13 +-
 .../apache/nifi/util/MockValidationContext.java    |   39 +-
 .../nifi/web/api/dto/AffectedComponentDTO.java     |    5 +
 .../nifi/web/api/dto/AsynchronousRequestDTO.java   |  116 ++
 .../web/api/dto/ComponentValidationResultDTO.java  |   56 +
 .../nifi/web/api/dto/ParameterContextDTO.java      |   83 ++
 .../web/api/dto/ParameterContextReferenceDTO.java  |   22 +-
 .../api/dto/ParameterContextUpdateRequestDTO.java  |   47 +
 .../web/api/dto/ParameterContextUpdateStepDTO.java |   20 +-
 .../dto/ParameterContextValidationRequestDTO.java  |   47 +
 .../dto/ParameterContextValidationResultsDTO.java  |   22 +-
 .../api/dto/ParameterContextValidationStepDTO.java |   20 +-
 .../org/apache/nifi/web/api/dto/ParameterDTO.java  |   82 ++
 .../apache/nifi/web/api/dto/ProcessGroupDTO.java   |   10 +
 .../org/apache/nifi/web/api/dto/TemplateDTO.java   |    2 +-
 ...gistryUpdateStepDTO.java => UpdateStepDTO.java} |    9 +-
 .../api/dto/VariableRegistryUpdateRequestDTO.java  |   78 +-
 .../web/api/dto/VariableRegistryUpdateStepDTO.java |   36 +-
 .../nifi/web/api/dto/flow/ProcessGroupFlowDTO.java |   10 +
 .../web/api/entity/AffectedComponentEntity.java    |    5 +
 ...y.java => ComponentValidationResultEntity.java} |   24 +-
 ....java => ComponentValidationResultsEntity.java} |   28 +-
 ...nentEntity.java => ParameterContextEntity.java} |   29 +-
 .../ParameterContextUpdateRequestEntity.java       |   47 +
 .../ParameterContextValidationRequestEntity.java   |   47 +
 .../web/api/entity/ParameterContextsEntity.java    |   54 +
 ...edComponentEntity.java => ParameterEntity.java} |   35 +-
 .../authorization/FileAccessPolicyProvider.java    |   67 +-
 .../nifi/authorization/FileUserGroupProvider.java  |    2 +-
 .../FileAccessPolicyProviderTest.java              |   71 +-
 .../nifi/authorization/FileAuthorizerTest.java     |   46 +-
 .../authorization/FileUserGroupProviderTest.java   |   25 +-
 .../StandardAuthorizerConfigurationContext.java    |    3 +-
 .../authorization/resource/ResourceFactory.java    |   30 +
 .../nifi/authorization/resource/ResourceType.java  |    3 +-
 ...CompositeConfigurableUserGroupProviderTest.java |   15 +-
 .../CompositeUserGroupProviderTest.java            |    3 +-
 .../CompositeUserGroupProviderTestBase.java        |    3 +-
 .../StandardManagedAuthorizerTest.java             |    4 +-
 .../http/StandardHttpResponseMapper.java           |   18 +-
 .../endpoints/ParameterContextEndpointMerger.java  |   50 +
 .../http/endpoints/ParameterContextMerger.java     |  155 +++
 .../ParameterContextUpdateEndpointMerger.java      |   87 ++
 .../ParameterContextValidationMerger.java          |  156 +++
 .../endpoints/ParameterContextsEndpointMerger.java |   72 ++
 .../coordination/node/NodeClusterCoordinator.java  |    7 +-
 .../apache/nifi/cluster/manager/StatusMerger.java  |    4 +
 .../flow/TestPopularVoteFlowElection.java          |   60 +-
 .../nifi-framework/nifi-framework-core-api/pom.xml |   22 +-
 .../nifi/controller/AbstractComponentNode.java     |  348 +++++-
 .../org/apache/nifi/controller/ComponentNode.java  |   49 +-
 .../org/apache/nifi/controller/ProcessorNode.java  |    7 +-
 .../nifi/controller/PropertyConfiguration.java     |  125 ++
 .../nifi/controller/ValidationContextFactory.java  |   11 +-
 .../apache/nifi/controller/flow/FlowManager.java   |    7 +
 .../java/org/apache/nifi/groups/ProcessGroup.java  |   39 +-
 .../apache/nifi/parameter/ParameterContext.java    |  100 ++
 .../nifi/parameter/ParameterContextManager.java    |   20 +-
 .../nifi/parameter/ParameterReferenceManager.java  |   72 ++
 .../apache/nifi/registry/flow/FlowRegistry.java    |    5 +-
 .../nifi/controller/TestAbstractComponentNode.java |   30 +-
 .../org/apache/nifi/controller/FlowController.java |    9 +-
 .../nifi/controller/StandardFlowSnippet.java       |   10 +
 .../nifi/controller/StandardFlowSynchronizer.java  |  114 +-
 .../nifi/controller/StandardProcessorNode.java     |   31 +-
 .../org/apache/nifi/controller/TemplateUtils.java  |   66 ++
 .../nifi/controller/flow/StandardFlowManager.java  |   38 +-
 .../reporting/AbstractReportingTaskNode.java       |    6 +
 .../reporting/StandardReportingContext.java        |    8 +-
 .../reporting/StandardReportingTaskNode.java       |    9 +-
 .../scheduling/StandardProcessScheduler.java       |    9 +-
 .../serialization/FlowFromDOMFactory.java          |   73 +-
 .../serialization/StandardFlowSerializer.java      |   62 +-
 .../service/ControllerServiceLoader.java           |   29 +-
 .../service/StandardConfigurationContext.java      |   10 +-
 .../service/StandardControllerServiceNode.java     |   21 +-
 .../service/StandardControllerServiceProvider.java |   47 +-
 .../manager/StandardStateManagerProvider.java      |   51 +-
 .../nifi/fingerprint/FingerprintFactory.java       |   75 +-
 .../apache/nifi/groups/StandardProcessGroup.java   |  167 ++-
 .../nifi/parameter/StandardParameterContext.java   |  306 +++++
 .../parameter/StandardParameterContextManager.java |   60 +
 .../StandardParameterReferenceManager.java         |   98 ++
 .../nifi/persistence/TemplateDeserializer.java     |    6 +-
 .../nifi/processor/StandardProcessContext.java     |   16 +-
 .../nifi/processor/StandardValidationContext.java  |   92 +-
 .../StandardValidationContextFactory.java          |   18 +-
 .../nifi/registry/flow/RestBasedFlowRegistry.java  |    9 +-
 .../flow/mapping/NiFiRegistryFlowMapper.java       |   43 +-
 .../src/main/resources/FlowConfiguration.xsd       |   29 +
 .../apache/nifi/controller/TestFlowController.java |   14 +-
 .../nifi/controller/TestStandardProcessorNode.java |   30 +-
 .../queue/clustered/LoadBalancedQueueIT.java       |    2 +-
 .../controller/service/mock/MockProcessGroup.java  |   26 +
 .../local/TestWriteAheadLocalStateProvider.java    |   28 +-
 .../zookeeper/TestZooKeeperStateProvider.java      |   18 +-
 .../cs/ControllerServiceReferencingProcessor.java  |   22 +-
 .../nifi/integration/parameters/ParametersIT.java  |  369 ++++++
 .../processor/ProcessorParameterTokenIT.java       |  211 ++++
 .../UpdateAttributeCreateOwnProperty.java          |   69 ++
 .../UpdateAttributeNoEL.java}                      |   40 +-
 .../UpdateAttributeWithEL.java}                    |   40 +-
 .../nifi/integration/versioned/ImportFlowIT.java   |    2 +-
 .../parameter/TestStandardParameterContext.java    |  291 +++++
 .../nifi/processor/TestStandardPropertyValue.java  |   35 +-
 .../nifi/remote/PeerDescriptionModifier.java       |   12 +-
 .../nifi/audit/ControllerServiceAuditor.java       |    5 +-
 .../org/apache/nifi/audit/ProcessorAuditor.java    |    2 +-
 .../apache/nifi/audit/ReportingTaskAuditor.java    |    2 +-
 .../nifi/authorization/AuthorizableLookup.java     |   14 +
 .../AuthorizeControllerServiceReference.java       |   14 +-
 .../authorization/AuthorizeParameterReference.java |  127 ++
 .../nifi/authorization/ComponentAuthorizable.java  |   13 +
 .../authorization/ProcessGroupAuthorizable.java    |    8 +
 .../nifi/authorization/SnippetAuthorizable.java    |    2 +-
 .../authorization/StandardAuthorizableLookup.java  |  108 +-
 .../org/apache/nifi/web/NiFiServiceFacade.java     |  109 +-
 .../apache/nifi/web/NiFiWebApiResourceConfig.java  |    1 +
 .../apache/nifi/web/StandardNiFiServiceFacade.java |  322 ++++-
 .../apache/nifi/web/api/ApplicationResource.java   |   45 +-
 .../nifi/web/api/ControllerServiceResource.java    |    3 +
 .../java/org/apache/nifi/web/api/FlowResource.java |   42 +-
 .../nifi/web/api/ParameterContextResource.java     | 1249 ++++++++++++++++++++
 .../apache/nifi/web/api/ProcessGroupResource.java  |  124 +-
 .../org/apache/nifi/web/api/ProcessorResource.java |    4 +-
 .../org/apache/nifi/web/api/SnippetResource.java   |   26 +-
 .../org/apache/nifi/web/api/VersionsResource.java  |   93 +-
 .../web/api/concurrent/AsyncRequestManager.java    |   22 +-
 .../web/api/concurrent/AsynchronousWebRequest.java |   42 +-
 .../nifi/web/api/concurrent/RequestManager.java    |   12 +-
 .../concurrent/StandardAsynchronousWebRequest.java |  104 +-
 .../web/api/concurrent/StandardUpdateStep.java     |   34 +-
 .../apache/nifi/web/api/concurrent/UpdateStep.java |   20 +-
 .../org/apache/nifi/web/api/dto/DtoFactory.java    |  174 ++-
 .../org/apache/nifi/web/api/dto/EntityFactory.java |   32 +
 .../web/controller/ControllerSearchService.java    |    2 +-
 .../nifi/web/controller/StandardSearchContext.java |   11 +-
 .../apache/nifi/web/dao/ParameterContextDAO.java   |   93 ++
 .../java/org/apache/nifi/web/dao/TemplateDAO.java  |    7 +-
 .../web/dao/impl/StandardControllerServiceDAO.java |   10 +
 .../web/dao/impl/StandardParameterContextDAO.java  |  255 ++++
 .../nifi/web/dao/impl/StandardProcessGroupDAO.java |   45 +
 .../nifi/web/dao/impl/StandardProcessorDAO.java    |    9 +
 .../nifi/web/dao/impl/StandardSnippetDAO.java      |    4 +-
 .../nifi/web/dao/impl/StandardTemplateDAO.java     |  123 +-
 .../util/ClusterReplicationComponentLifecycle.java |  108 +-
 .../apache/nifi/web/util/ComponentLifecycle.java   |    6 +-
 .../nifi/web/util/InvalidComponentAction.java      |   29 +-
 .../nifi/web/util/LocalComponentLifecycle.java     |  123 +-
 .../org/apache/nifi/web/util/SnippetUtils.java     |   12 +-
 .../src/main/resources/nifi-web-api-context.xml    |   16 +
 .../nifi-framework-bundle/nifi-framework/pom.xml   |    2 +-
 nifi-nar-bundles/nifi-framework-bundle/pom.xml     |    5 +
 .../ldap/tenants/LdapUserGroupProviderTest.java    |  202 ++--
 .../provenance/index/lucene/EventIndexTask.java    |    2 +-
 .../nifi/redis/state/ITRedisStateProvider.java     |    4 +-
 .../nifi/script/impl/ValidationContextAdapter.java |   14 +-
 .../TestSiteToSiteBulletinReportingTask.java       |   35 +-
 .../api/transformjson/TransformJSONResource.java   |    3 +-
 .../attributes/UpdateAttributeModelFactory.java    |   10 +-
 nifi-stateless/README.md                           |    6 +-
 .../nifi/stateless/core/ComponentFactory.java      |   26 +-
 .../core/StatelessConfigurationContext.java        |    8 +-
 .../core/StatelessControllerServiceLookup.java     |   20 +-
 .../apache/nifi/stateless/core/StatelessFlow.java  |   73 +-
 .../stateless/core/StatelessParameterContext.java  |  117 ++
 .../stateless/core/StatelessProcessContext.java    |  148 ++-
 .../stateless/core/StatelessProcessorWrapper.java  |    7 +-
 .../stateless/core/StatelessPropertyValue.java     |   26 +-
 .../stateless/core/StatelessValidationContext.java |   41 +-
 .../org/apache/nifi/stateless/core/BatchTest.java  |    6 +-
 .../apache/nifi/stateless/core/StreamingIT.java    |    4 +-
 312 files changed, 10627 insertions(+), 2357 deletions(-)

diff --git a/nifi-api/src/main/java/org/apache/nifi/components/AbstractConfigurableComponent.java b/nifi-api/src/main/java/org/apache/nifi/components/AbstractConfigurableComponent.java
index d0e5a3b..25ffae5 100644
--- a/nifi-api/src/main/java/org/apache/nifi/components/AbstractConfigurableComponent.java
+++ b/nifi-api/src/main/java/org/apache/nifi/components/AbstractConfigurableComponent.java
@@ -84,12 +84,14 @@ public abstract class AbstractConfigurableComponent implements ConfigurableCompo
         // goes through supported properties
         final Collection<ValidationResult> results = new ArrayList<>();
         final List<PropertyDescriptor> supportedDescriptors = getSupportedPropertyDescriptors();
+
         if (null != supportedDescriptors) {
             for (final PropertyDescriptor descriptor : supportedDescriptors) {
                 String value = context.getProperty(descriptor).getValue();
                 if (value == null) {
                     value = descriptor.getDefaultValue();
                 }
+
                 if (value == null && descriptor.isRequired()) {
                     String displayName = descriptor.getDisplayName();
                     ValidationResult.Builder builder = new ValidationResult.Builder().valid(false).input(null).subject(displayName != null ? displayName : descriptor.getName());
diff --git a/nifi-api/src/main/java/org/apache/nifi/components/ValidationContext.java b/nifi-api/src/main/java/org/apache/nifi/components/ValidationContext.java
index 444d1bd..95a8c09 100644
--- a/nifi-api/src/main/java/org/apache/nifi/components/ValidationContext.java
+++ b/nifi-api/src/main/java/org/apache/nifi/components/ValidationContext.java
@@ -16,13 +16,14 @@
  */
 package org.apache.nifi.components;
 
-import java.util.Map;
-
 import org.apache.nifi.context.PropertyContext;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.ControllerServiceLookup;
 import org.apache.nifi.expression.ExpressionLanguageCompiler;
 
+import java.util.Collection;
+import java.util.Map;
+
 public interface ValidationContext extends PropertyContext {
 
     /**
@@ -93,4 +94,16 @@ public interface ValidationContext extends PropertyContext {
      * @return the identifier of the ProcessGroup that the component being validated lives in
      */
     String getProcessGroupIdentifier();
+
+    /**
+     * Returns a Collection containing the names of all Parameters that are referenced by the property with the given name
+     * @return  a Collection containing the names of all Parameters that are referenced by the property with the given name
+     */
+    Collection<String> getReferencedParameters(String propertyName);
+
+    /**
+     * @param parameterName the name of the Parameter
+     * @return <code>true</code> if a Parameter with the given name is defined in the currently selected Parameter Context
+     */
+    boolean isParameterDefined(String parameterName);
 }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java b/nifi-api/src/main/java/org/apache/nifi/parameter/Parameter.java
similarity index 53%
copy from nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java
copy to nifi-api/src/main/java/org/apache/nifi/parameter/Parameter.java
index e9ac03b..70f61af 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java
+++ b/nifi-api/src/main/java/org/apache/nifi/parameter/Parameter.java
@@ -14,39 +14,43 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.attribute.expression.language;
+package org.apache.nifi.parameter;
 
+import java.util.Objects;
 
-import java.util.Map;
-
-import org.apache.nifi.expression.AttributeValueDecorator;
-import org.apache.nifi.processor.exception.ProcessException;
-
-public class EmptyPreparedQuery implements PreparedQuery {
-
+public class Parameter {
+    private final ParameterDescriptor descriptor;
     private final String value;
 
-    EmptyPreparedQuery(final String value) {
+    public Parameter(final ParameterDescriptor descriptor, final String value) {
+        this.descriptor = descriptor;
         this.value = value;
     }
 
-    @Override
-    public String evaluateExpressions(Map<String, String> valueLookup, AttributeValueDecorator decorator) throws ProcessException {
-        return value;
+    public ParameterDescriptor getDescriptor() {
+        return descriptor;
     }
 
-    @Override
-    public String evaluateExpressions(Map<String, String> attributes, AttributeValueDecorator decorator, Map<String, String> stateVariables) throws ProcessException {
+    public String getValue() {
         return value;
     }
 
     @Override
-    public boolean isExpressionLanguagePresent() {
-        return false;
+    public boolean equals(final Object o) {
+        if (this == o) {
+            return true;
+        }
+
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+
+        final Parameter parameter = (Parameter) o;
+        return Objects.equals(descriptor, parameter.descriptor) && Objects.equals(value, parameter.value);
     }
 
     @Override
-    public VariableImpact getVariableImpact() {
-        return VariableImpact.NEVER_IMPACTED;
+    public int hashCode() {
+        return Objects.hash(descriptor, value);
     }
 }
diff --git a/nifi-api/src/main/java/org/apache/nifi/parameter/ParameterDescriptor.java b/nifi-api/src/main/java/org/apache/nifi/parameter/ParameterDescriptor.java
new file mode 100644
index 0000000..bf785bf
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/parameter/ParameterDescriptor.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.parameter;
+
+import java.util.Objects;
+
+/**
+ * An immutable class that holds information about a Parameter. Parameters are constructed through use of the {@link ParameterDescriptor.Builder} class.
+ */
+public class ParameterDescriptor {
+    private final String name;
+    private final String description;
+    private final boolean sensitive;
+
+    private ParameterDescriptor(final Builder builder) {
+        this.name = builder.name;
+        this.description = builder.description;
+        this.sensitive = builder.sensitive;
+    }
+
+    /**
+     * @return the name of the parameter
+     */
+    public String getName() {
+        return name;
+    }
+
+    /**
+     * @return a description of the parameter
+     */
+    public String getDescription() {
+        return description;
+    }
+
+    /**
+     * @return whether or not the parameter is considered sensitive.
+     */
+    public boolean isSensitive() {
+        return sensitive;
+    }
+
+
+    @Override
+    public boolean equals(final Object o) {
+        if (this == o){
+            return true;
+        }
+
+        if (o == null || getClass() != o.getClass()){
+            return false;
+        }
+
+
+        final ParameterDescriptor other = (ParameterDescriptor) o;
+        return Objects.equals(this.name, other.name);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(name);
+    }
+
+
+    public static class Builder {
+        private String name;
+        private String description;
+        private boolean sensitive;
+
+        public Builder name(final String name) {
+            Objects.requireNonNull(name);
+
+            this.name = name.trim();
+            return this;
+        }
+
+        public Builder description(final String description) {
+            this.description = description == null ? null : description.trim();
+            return this;
+        }
+
+        public Builder sensitive(final boolean sensitive) {
+            this.sensitive = sensitive;
+            return this;
+        }
+
+        public Builder from(final ParameterDescriptor descriptor) {
+            name(descriptor.getName());
+            description(descriptor.getDescription());
+            sensitive(descriptor.isSensitive());
+            return this;
+        }
+
+        public ParameterDescriptor build() {
+            if (name == null) {
+                throw new IllegalStateException("Must specify Parameter Name");
+            }
+
+            return new ParameterDescriptor(this);
+        }
+    }
+}
diff --git a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/NotificationServiceManager.java b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/NotificationServiceManager.java
index 6e91751..6e2fa35 100644
--- a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/NotificationServiceManager.java
+++ b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/NotificationServiceManager.java
@@ -16,27 +16,7 @@
  */
 package org.apache.nifi.bootstrap;
 
-import java.io.BufferedInputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-
+import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
 import org.apache.nifi.bootstrap.notification.NotificationContext;
 import org.apache.nifi.bootstrap.notification.NotificationInitializationContext;
@@ -57,6 +37,26 @@ import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 import org.xml.sax.SAXException;
 
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import java.io.BufferedInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
 public class NotificationServiceManager {
     private static final Logger logger = LoggerFactory.getLogger(NotificationServiceManager.class);
     private final Map<String, ConfiguredNotificationService> servicesById = new HashMap<>();
@@ -256,7 +256,7 @@ public class NotificationServiceManager {
                     configuredValue = fullPropDescriptor.getDefaultValue();
                 }
 
-                return new StandardPropertyValue(configuredValue, null, variableRegistry);
+                return new StandardPropertyValue(configuredValue, null, ParameterLookup.EMPTY, variableRegistry);
             }
 
             @Override
@@ -376,7 +376,7 @@ public class NotificationServiceManager {
                         value = descriptor.getDefaultValue();
                     }
 
-                    return new StandardPropertyValue(value, null, variableRegistry);
+                    return new StandardPropertyValue(value, null, ParameterLookup.EMPTY, variableRegistry);
                 }
 
                 @Override
diff --git a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/NotificationValidationContext.java b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/NotificationValidationContext.java
index 6d3ef53..8f7eda0 100644
--- a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/NotificationValidationContext.java
+++ b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/NotificationValidationContext.java
@@ -16,11 +16,7 @@
  */
 package org.apache.nifi.bootstrap.notification;
 
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-
+import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.attribute.expression.language.Query;
 import org.apache.nifi.attribute.expression.language.Query.Range;
 import org.apache.nifi.attribute.expression.language.StandardExpressionLanguageCompiler;
@@ -33,6 +29,13 @@ import org.apache.nifi.controller.ControllerServiceLookup;
 import org.apache.nifi.expression.ExpressionLanguageCompiler;
 import org.apache.nifi.registry.VariableRegistry;
 
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
 public class NotificationValidationContext implements ValidationContext {
     private final NotificationContext context;
     private final Map<String, Boolean> expressionLanguageSupported;
@@ -52,13 +55,13 @@ public class NotificationValidationContext implements ValidationContext {
 
     @Override
     public PropertyValue newPropertyValue(final String rawValue) {
-        return new StandardPropertyValue(rawValue, null, variableRegistry);
+        return new StandardPropertyValue(rawValue, null, ParameterLookup.EMPTY, variableRegistry);
     }
 
     @Override
     public ExpressionLanguageCompiler newExpressionLanguageCompiler() {
 
-        return new StandardExpressionLanguageCompiler(null);
+        return new StandardExpressionLanguageCompiler(null, ParameterLookup.EMPTY);
     }
 
     @Override
@@ -120,4 +123,14 @@ public class NotificationValidationContext implements ValidationContext {
     public String getProcessGroupIdentifier() {
         return null;
     }
+
+    @Override
+    public Collection<String> getReferencedParameters(final String propertyName) {
+        return Collections.emptyList();
+    }
+
+    @Override
+    public boolean isParameterDefined(final String parameterName) {
+        return false;
+    }
 }
diff --git a/nifi-commons/nifi-expression-language/pom.xml b/nifi-commons/nifi-expression-language/pom.xml
index a20ce36..606bd86 100644
--- a/nifi-commons/nifi-expression-language/pom.xml
+++ b/nifi-commons/nifi-expression-language/pom.xml
@@ -49,7 +49,7 @@
                         <exclude>src/test/resources/json/address-book.json</exclude>
                     </excludes>
                 </configuration>
-            </plugin>            
+            </plugin>
         </plugins>
     </build>
     <dependencies>
@@ -65,6 +65,11 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-parameter</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
             <version>1.10.0-SNAPSHOT</version>
         </dependency>
diff --git a/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g b/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g
index d1b22e9..3b8e65f 100644
--- a/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g
+++ b/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g
@@ -66,7 +66,10 @@ lexer grammar AttributeExpressionLexer;
 
 // PUNCTUATION & SPECIAL CHARACTERS
 WHITESPACE : (' '|'\t'|'\n'|'\r')+ { $channel = HIDDEN; };
-COMMENT : '#' ( ~('\n') )* '\n' { $channel = HIDDEN; };
+COMMENT : '#' ( ~('{') ) ( ~('\n') )* '\n' { $channel = HIDDEN; };
+
+// PARAMETERS
+PARAMETER_REFERENCE_START : '#{';
 
 DOLLAR : '$';
 LPAREN	: '(';
diff --git a/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g b/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g
index 0c63c55..2728e36 100644
--- a/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g
+++ b/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g
@@ -29,6 +29,7 @@ tokens {
 	EXPRESSION;
 	MULTI_ATTRIBUTE_REFERENCE;
 	QUOTED_ATTR_NAME;
+	PARAMETER_REFERENCE;
 }
 
 @header {
@@ -101,7 +102,9 @@ stringFunctionRef : zeroArgString | oneArgString | twoArgString | fiveArgString;
 booleanFunctionRef : zeroArgBool | oneArgBool | multiArgBool;
 numberFunctionRef : zeroArgNum | oneArgNum | oneOrTwoArgNum | zeroOrOneOrTwoArgNum;
 
-anyArg : WHOLE_NUMBER | DECIMAL | numberFunctionRef | STRING_LITERAL | zeroArgString | oneArgString | twoArgString | fiveArgString | booleanLiteral | zeroArgBool | oneArgBool | multiArgBool | expression;
+anyArg : WHOLE_NUMBER | DECIMAL | numberFunctionRef | STRING_LITERAL | zeroArgString | oneArgString | twoArgString | fiveArgString | booleanLiteral | zeroArgBool | oneArgBool | multiArgBool
+                | expression | parameterReference;
+
 stringArg : STRING_LITERAL | zeroArgString | oneArgString | twoArgString | expression;
 functionRef : stringFunctionRef | booleanFunctionRef | numberFunctionRef;
 
@@ -133,10 +136,15 @@ oneArgStandaloneFunction : ((TO_LITERAL | MATH | GET_STATE_VALUE)^ LPAREN! anyAr
                            (HOSTNAME^ LPAREN! booleanLiteral RPAREN!);
 standaloneFunction : zeroArgStandaloneFunction | oneArgStandaloneFunction;
 
-attributeRefOrFunctionCall	: (attributeRef | standaloneFunction);
+attributeRefOrFunctionCall	: (attributeRef | standaloneFunction | parameterReference);
+
+referenceOrFunction : DOLLAR LBRACE attributeRefOrFunctionCall (COLON functionCall)* RBRACE ->
+                      	^(EXPRESSION attributeRefOrFunctionCall functionCall*);
+
+parameterReference : PARAMETER_REFERENCE_START ATTRIBUTE_NAME RBRACE ->
+    ^(PARAMETER_REFERENCE ATTRIBUTE_NAME);
 
-expression : DOLLAR LBRACE attributeRefOrFunctionCall (COLON functionCall)* RBRACE ->
-	^(EXPRESSION attributeRefOrFunctionCall functionCall*);
+expression : referenceOrFunction;
 
 query : expression EOF ->
 	^(QUERY expression);
\ No newline at end of file
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 52fcb1f..74741b1 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
@@ -17,13 +17,12 @@
 
 package org.apache.nifi.attribute.expression.language;
 
-import java.util.Map;
-import java.util.Set;
-
 import org.antlr.runtime.tree.Tree;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.expression.AttributeValueDecorator;
 
+import java.util.Set;
+
 public class CompiledExpression implements Expression {
     private final Evaluator<?> rootEvaluator;
     private final Tree tree;
@@ -54,7 +53,7 @@ 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(tree, rootEvaluator, expression, variables, decorator, stateVariables);
+    public String evaluate(final EvaluationContext evaluationContext, final AttributeValueDecorator decorator) {
+        return Query.evaluateExpression(tree, rootEvaluator, expression, evaluationContext, decorator);
     }
 }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java
index e9ac03b..d746e7b 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java
@@ -17,8 +17,6 @@
 package org.apache.nifi.attribute.expression.language;
 
 
-import java.util.Map;
-
 import org.apache.nifi.expression.AttributeValueDecorator;
 import org.apache.nifi.processor.exception.ProcessException;
 
@@ -31,12 +29,7 @@ public class EmptyPreparedQuery implements PreparedQuery {
     }
 
     @Override
-    public String evaluateExpressions(Map<String, String> valueLookup, AttributeValueDecorator decorator) throws ProcessException {
-        return value;
-    }
-
-    @Override
-    public String evaluateExpressions(Map<String, String> attributes, AttributeValueDecorator decorator, Map<String, String> stateVariables) throws ProcessException {
+    public String evaluateExpressions(final EvaluationContext evaluationContext, final AttributeValueDecorator decorator) throws ProcessException {
         return value;
     }
 
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/EvaluationContext.java
similarity index 50%
copy from nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/BooleanLiteralEvaluator.java
copy to nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EvaluationContext.java
index cd04eec..83e67de 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/EvaluationContext.java
@@ -14,32 +14,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.attribute.expression.language.evaluation.literals;
+package org.apache.nifi.attribute.expression.language;
 
-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.EvaluatorState;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
+import org.apache.nifi.parameter.Parameter;
 
-public class BooleanLiteralEvaluator extends BooleanEvaluator {
+import java.util.Set;
 
-    private final boolean value;
+public interface EvaluationContext {
+    /**
+     * Returns the name of an attribute, variable, environment variable, or system variable that can be referenced in the Expression Language
+     * @param name the name of the attribute, variable, etc.
+     * @return the value assigned to the attribute, variable, etc. or <code>null</code> if no such value exists
+     */
+    String getExpressionValue(String name);
 
-    public BooleanLiteralEvaluator(final boolean value) {
-        this.value = value;
-    }
+    /**
+     * Returns the names of all attributes, variables, etc. that can be used in Expression Language
+     * @return the names of all keys that can be used in Expression Language
+     */
+    Set<String> getExpressionKeys();
 
-    @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        return new BooleanQueryResult(value);
-    }
+    String getState(String key);
 
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return null;
-    }
+    Parameter getParameter(String parameterName);
 
+    EvaluatorState getEvaluatorState();
 }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Expression.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Expression.java
index d229986..188c503 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Expression.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Expression.java
@@ -17,18 +17,15 @@
 
 package org.apache.nifi.attribute.expression.language;
 
-import java.util.Map;
-
 import org.apache.nifi.expression.AttributeValueDecorator;
 
 public interface Expression {
     /**
      * Evaluates this Expression against the given variables, attribute decorator, and state variables
      *
-     * @param variables variables to be evaluated
+     * @param evaluationContext the context used to evaluate the Expression
      * @param decorator decorator to decorate variable values
-     * @param stateVariables state variables to include in evaluation
      * @return the evaluated value
      */
-    String evaluate(Map<String, String> variables, AttributeValueDecorator decorator, Map<String, String> stateVariables);
+    String evaluate(EvaluationContext evaluationContext, AttributeValueDecorator decorator);
 }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java
index 8ef996b..9a3ec32 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java
@@ -17,8 +17,6 @@
 package org.apache.nifi.attribute.expression.language;
 
 
-import java.util.Map;
-
 import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageException;
 import org.apache.nifi.expression.AttributeValueDecorator;
 import org.apache.nifi.processor.exception.ProcessException;
@@ -40,12 +38,7 @@ public class InvalidPreparedQuery implements PreparedQuery {
     }
 
     @Override
-    public String evaluateExpressions(final Map<String,String> valueLookup, final AttributeValueDecorator decorator) throws ProcessException {
-        throw new AttributeExpressionLanguageException("Invalid Expression: " + query + " due to " + explanation);
-    }
-
-    @Override
-    public String evaluateExpressions( Map<String, String> valueLookup, AttributeValueDecorator decorator, Map<String, String> stateVariables) throws ProcessException {
+    public String evaluateExpressions(final EvaluationContext evaluationContext, final AttributeValueDecorator decorator) throws ProcessException {
         throw new AttributeExpressionLanguageException("Invalid Expression: " + query + " due to " + explanation);
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/ParameterExpression.java
similarity index 63%
copy from nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java
copy to nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/ParameterExpression.java
index 74dafce..914a4d5 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/ParameterExpression.java
@@ -14,22 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.nifi.attribute.expression.language;
 
-import java.util.Map;
-
 import org.apache.nifi.expression.AttributeValueDecorator;
+import org.apache.nifi.parameter.Parameter;
 
-public class StringLiteralExpression implements Expression {
-    private final String value;
+public class ParameterExpression implements Expression {
+    private final String parameterName;
 
-    public StringLiteralExpression(final String value) {
-        this.value = value;
+    public ParameterExpression(final String parameterName) {
+        this.parameterName = parameterName;
     }
 
     @Override
-    public String evaluate(Map<String, String> variables, AttributeValueDecorator decorator, Map<String, String> stateVariables) {
-        return value;
+    public String evaluate(final EvaluationContext evaluationContext, final AttributeValueDecorator decorator) {
+        final Parameter parameter = evaluationContext.getParameter(parameterName);
+        return parameter == null || parameter.getDescriptor().isSensitive() ? null : parameter.getValue();
     }
 }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java
index c51656d..7bdd287 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java
@@ -17,16 +17,12 @@
 package org.apache.nifi.attribute.expression.language;
 
 
-import java.util.Map;
-
 import org.apache.nifi.expression.AttributeValueDecorator;
 import org.apache.nifi.processor.exception.ProcessException;
 
 public interface PreparedQuery {
 
-    String evaluateExpressions(Map<String, String> valueLookup, AttributeValueDecorator decorator) throws ProcessException;
-
-    String evaluateExpressions(final Map<String, String> valueLookup, final AttributeValueDecorator decorator, final Map<String, String> stateVariables) throws ProcessException;
+    String evaluateExpressions(EvaluationContext evaluationContext, AttributeValueDecorator decorator) throws ProcessException;
 
     boolean isExpressionLanguagePresent();
 
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 b7c5259..e0f4cf0 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,16 +18,23 @@ 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.EvaluatorState;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 import org.apache.nifi.attribute.expression.language.evaluation.selection.AttributeEvaluator;
 import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
 import org.apache.nifi.expression.AttributeValueDecorator;
+import org.apache.nifi.parameter.ExpressionLanguageAwareParameterParser;
+import org.apache.nifi.parameter.ParameterLookup;
+import org.apache.nifi.parameter.ParameterParser;
+import org.apache.nifi.parameter.ParameterReference;
+import org.apache.nifi.parameter.ParameterToken;
+import org.apache.nifi.parameter.ParameterTokenList;
 import org.apache.nifi.processor.exception.ProcessException;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -85,12 +92,21 @@ public class Query {
         return expressions;
     }
 
+
     public static List<Range> extractExpressionRanges(final String value) throws AttributeExpressionLanguageParsingException {
+        return extractExpressionRanges(value, false);
+    }
+
+    public static List<Range> extractEscapedRanges(final String value) throws AttributeExpressionLanguageParsingException {
+        return extractExpressionRanges(value, true);
+    }
+
+    private static List<Range> extractExpressionRanges(final String value, final boolean extractEscapeSequences) throws AttributeExpressionLanguageParsingException {
         final List<Range> ranges = new ArrayList<>();
         char lastChar = 0;
         int embeddedCount = 0;
         int expressionStart = -1;
-        boolean oddDollarCount = false;
+        int dollarCount = 0;
         int backslashCount = 0;
 
         charLoop:
@@ -108,9 +124,10 @@ public class Query {
                 }
 
                 if (c == '{') {
-                    if (oddDollarCount && lastChar == '$') {
+                    final boolean evenDollarCount = dollarCount % 2 == 0;
+                    if ((evenDollarCount == extractEscapeSequences) && lastChar == '$') {
                         if (embeddedCount == 0) {
-                            expressionStart = i - 1;
+                            expressionStart = i - (extractEscapeSequences ? dollarCount : 1);
                         }
                     }
 
@@ -137,11 +154,11 @@ public class Query {
                         expressionStart = -1;
                     }
                 } else if (c == '$') {
-                    oddDollarCount = !oddDollarCount;
+                    dollarCount++;
                 } else if (c == '\\') {
                     backslashCount++;
                 } else {
-                    oddDollarCount = false;
+                    dollarCount = 0;
                 }
 
                 lastChar = c;
@@ -201,10 +218,11 @@ public class Query {
         return -1;
     }
 
-    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 {
+    static String evaluateExpression(final Tree tree, final Evaluator<?> rootEvaluator, final String queryText, final EvaluationContext evaluationContext, final AttributeValueDecorator decorator)
+                throws ProcessException {
+
         Query query = new Query(queryText, tree, rootEvaluator);
-        final Object evaluated = query.evaluate(valueMap, stateVariables).getValue();
+        final Object evaluated = query.evaluate(evaluationContext).getValue();
         if (evaluated == null) {
             return null;
         }
@@ -213,17 +231,18 @@ public class Query {
         return decorator == null ? value : decorator.decorate(value);
     }
 
-    static String evaluateExpressions(final String rawValue, Map<String, String> expressionMap, final AttributeValueDecorator decorator, final Map<String, String> stateVariables)
-            throws ProcessException {
-        return Query.prepare(rawValue).evaluateExpressions(expressionMap, decorator, stateVariables);
+    static String evaluateExpressions(final String rawValue, Map<String, String> expressionMap, final AttributeValueDecorator decorator, final Map<String, String> stateVariables,
+                                      final ParameterLookup parameterLookup) throws ProcessException {
+        return Query.prepare(rawValue).evaluateExpressions(new StandardEvaluationContext(expressionMap, stateVariables, parameterLookup), decorator);
     }
 
-    static String evaluateExpressions(final String rawValue, final Map<String, String> valueLookup) throws ProcessException {
-        return evaluateExpressions(rawValue, valueLookup, null);
+    static String evaluateExpressions(final String rawValue, final Map<String, String> valueLookup, final ParameterLookup parameterLookup) throws ProcessException {
+        return evaluateExpressions(rawValue, valueLookup, null, parameterLookup);
     }
 
-    static String evaluateExpressions(final String rawValue, final Map<String, String> valueLookup, final AttributeValueDecorator decorator) throws ProcessException {
-        return Query.prepare(rawValue).evaluateExpressions(valueLookup, decorator);
+    static String evaluateExpressions(final String rawValue, final Map<String, String> valueLookup, final AttributeValueDecorator decorator, final ParameterLookup parameterLookup)
+            throws ProcessException {
+        return Query.prepare(rawValue).evaluateExpressions(new StandardEvaluationContext(valueLookup, Collections.emptyMap(), parameterLookup), decorator);
     }
 
 
@@ -279,19 +298,61 @@ public class Query {
     }
 
 
+    public static PreparedQuery prepareWithParametersPreEvaluated(final String query) throws AttributeExpressionLanguageParsingException {
+        return prepare(query, true);
+    }
+
     public static PreparedQuery prepare(final String query) throws AttributeExpressionLanguageParsingException {
-        if (query == null) {
+        return prepare(query, false);
+    }
+
+    private static PreparedQuery prepare(final String rawQuery, final boolean escapeParameterReferences) throws AttributeExpressionLanguageParsingException {
+        if (rawQuery == null) {
             return new EmptyPreparedQuery(null);
         }
 
+        final ParameterParser parameterParser = new ExpressionLanguageAwareParameterParser();
+
+        final String query;
+        if (escapeParameterReferences) {
+            query = parameterParser.parseTokens(rawQuery).escape();
+        } else {
+            query = rawQuery;
+        }
+
         final List<Range> ranges = extractExpressionRanges(query);
 
         if (ranges.isEmpty()) {
-            // While in the other cases below, we are simply replacing "$$" with "$", we have to do this
-            // a bit differently. We want to treat $$ as an escaped $ only if it immediately precedes the
-            // start of an Expression, which is the case below. Here, we did not detect the start of an Expression
-            // and as such as must use the #unescape method instead of a simple replace() function.
-            return new EmptyPreparedQuery(unescape(query));
+            final List<Expression> expressions = new ArrayList<>();
+
+            final List<Range> escapedRanges = extractEscapedRanges(query);
+            int lastIndex = 0;
+            for (final Range range : escapedRanges) {
+                final String treeText = unescapeLeadingDollarSigns(query.substring(range.getStart(), range.getEnd() + 1));
+
+                if (range.getStart() > lastIndex) {
+                    String substring = unescapeLeadingDollarSigns(query.substring(lastIndex, range.getStart()));
+                    addLiteralsAndParameters(parameterParser, substring, expressions);
+                }
+
+                addLiteralsAndParameters(parameterParser, treeText, expressions);
+            }
+
+            if (escapedRanges.isEmpty()) {
+                addLiteralsAndParameters(parameterParser, query, expressions);
+            } else {
+                final Range lastRange = escapedRanges.get(escapedRanges.size() - 1);
+                if (lastRange.getEnd() + 1 < query.length()) {
+                    final String treeText = unescapeLeadingDollarSigns(query.substring(lastRange.getEnd() + 1));
+                    addLiteralsAndParameters(parameterParser, treeText, expressions);
+                }
+            }
+
+            if (expressions.isEmpty()) {
+                return new EmptyPreparedQuery(query);
+            }
+
+            return new StandardPreparedQuery(expressions);
         }
 
         final ExpressionCompiler compiler = new ExpressionCompiler();
@@ -314,7 +375,7 @@ public class Query {
                         substring = unescapeTrailingDollarSigns(substring, false);
                     }
 
-                    expressions.add(new StringLiteralExpression(substring));
+                    addLiteralsAndParameters(parameterParser, substring, expressions);
                 }
 
                 expressions.add(compiledExpression);
@@ -325,7 +386,7 @@ public class Query {
             final Range lastRange = ranges.get(ranges.size() - 1);
             if (lastRange.getEnd() + 1 < query.length()) {
                 final String treeText = unescapeLeadingDollarSigns(query.substring(lastRange.getEnd() + 1));
-                expressions.add(new StringLiteralExpression(treeText));
+                addLiteralsAndParameters(parameterParser, treeText, expressions);
             }
 
             return new StandardPreparedQuery(expressions);
@@ -334,6 +395,43 @@ public class Query {
         }
     }
 
+    private static void addLiteralsAndParameters(final ParameterParser parser, final String input, final List<Expression> expressions) {
+        final ParameterTokenList references = parser.parseTokens(input);
+        int index = 0;
+
+        ParameterToken lastReference = null;
+        for (final ParameterToken token : references) {
+            if (token.isEscapeSequence()) {
+                expressions.add(new StringLiteralExpression(token.getValue(ParameterLookup.EMPTY)));
+                index = token.getEndOffset() + 1;
+                lastReference = token;
+                continue;
+            }
+
+            final int start = token.getStartOffset();
+
+            if (start > index) {
+                expressions.add(new StringLiteralExpression(input.substring(index, start)));
+            }
+
+            if (token.isParameterReference()) {
+                final ParameterReference parameterReference = (ParameterReference) token;
+                expressions.add(new ParameterExpression(parameterReference.getParameterName()));
+            } else {
+                expressions.add(new StringLiteralExpression(token.getValue(ParameterLookup.EMPTY)));
+            }
+
+            index = token.getEndOffset() + 1;
+            lastReference = token;
+        }
+
+        if (lastReference == null) {
+            expressions.add(new StringLiteralExpression(input));
+        } else if (input.length() > lastReference.getEndOffset() + 1) {
+            expressions.add(new StringLiteralExpression(input.substring(lastReference.getEndOffset() + 1)));
+        }
+    }
+
     public static Query compile(final String query) throws AttributeExpressionLanguageParsingException {
         try {
             final ExpressionCompiler compiler = new ExpressionCompiler();
@@ -351,20 +449,12 @@ public class Query {
         return evaluator.getResultType();
     }
 
-    QueryResult<?> evaluate(final Map<String, String> map) {
-        return evaluate(map, null);
-    }
-
-    QueryResult<?> evaluate(final Map<String, String> attributes, final Map<String, String> stateMap) {
+    QueryResult<?> evaluate(final EvaluationContext evaluationContext) {
         if (evaluated.getAndSet(true)) {
             throw new IllegalStateException("A Query cannot be evaluated more than once");
         }
-        if (stateMap != null) {
-            AttributesAndState attributesAndState = new AttributesAndState(attributes, stateMap);
-            return evaluator.evaluate(attributesAndState, context);
-        } else {
-            return evaluator.evaluate(attributes, context);
-        }
+
+        return evaluator.evaluate(evaluationContext);
     }
 
 
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardAttributeExpression.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardAttributeExpression.java
index a7ed184..983392e 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardAttributeExpression.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardAttributeExpression.java
@@ -19,22 +19,23 @@ package org.apache.nifi.attribute.expression.language;
 import org.apache.nifi.expression.AttributeExpression;
 import org.apache.nifi.expression.AttributeValueDecorator;
 import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.registry.VariableRegistry;
 
+import java.util.Collections;
+
 public class StandardAttributeExpression implements AttributeExpression {
 
     private final Query query;
     private final VariableRegistry variableRegistry;
+    private final ParameterLookup parameterLookup;
 
-    public StandardAttributeExpression(final Query query) {
-        this.query = query;
-        this.variableRegistry = null;
-    }
 
-    public StandardAttributeExpression(final Query query, final VariableRegistry variableRegistry) {
+    public StandardAttributeExpression(final Query query, final VariableRegistry variableRegistry, final ParameterLookup parameterLookup) {
         this.query = query;
         this.variableRegistry = variableRegistry;
+        this.parameterLookup = parameterLookup;
     }
 
     @Override
@@ -60,7 +61,9 @@ public class StandardAttributeExpression implements AttributeExpression {
     @Override
     public String evaluate(final FlowFile flowFile, final AttributeValueDecorator decorator) throws ProcessException {
         final ValueLookup lookup = new ValueLookup(variableRegistry, flowFile);
-        final Object evaluationResult = query.evaluate(lookup).getValue();
+        final EvaluationContext evaluationContext = new StandardEvaluationContext(lookup, Collections.emptyMap(), parameterLookup);
+
+        final Object evaluationResult = query.evaluate(evaluationContext).getValue();
         if (evaluationResult == null) {
             return "";
         }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardEvaluationContext.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardEvaluationContext.java
new file mode 100644
index 0000000..c93efa2
--- /dev/null
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardEvaluationContext.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.attribute.expression.language;
+
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterLookup;
+import org.apache.nifi.registry.VariableRegistry;
+
+import java.util.Map;
+import java.util.Set;
+
+public class StandardEvaluationContext implements EvaluationContext {
+    private final ValueLookup valueLookup;
+    private final Map<String, String> stateMap;
+    private final ParameterLookup parameterLookup;
+    private final EvaluatorState evaluatorState = new EvaluatorState();
+
+    public StandardEvaluationContext(final Map<String, String> variables) {
+        this(variables, null, ParameterLookup.EMPTY);
+    }
+
+    public StandardEvaluationContext(final Map<String, String> variables, final Map<String, String> stateMap, final ParameterLookup parameterLookup) {
+        this(new ValueLookup(VariableRegistry.ENVIRONMENT_SYSTEM_REGISTRY, null, variables), stateMap, parameterLookup);
+    }
+
+    public StandardEvaluationContext(final ValueLookup valueLookup, final Map<String, String> stateMap, final ParameterLookup parameterLookup) {
+        this.valueLookup = valueLookup;
+        this.stateMap = stateMap;
+        this.parameterLookup = parameterLookup;
+    }
+
+    @Override
+    public String getExpressionValue(final String name) {
+        return valueLookup.get(name);
+    }
+
+    @Override
+    public Set<String> getExpressionKeys() {
+        return valueLookup.getKeysAddressableByMultiMatch();
+    }
+
+    @Override
+    public String getState(final String key) {
+        return stateMap.get(key);
+    }
+
+    @Override
+    public Parameter getParameter(final String parameterName) {
+        return parameterLookup.getParameter(parameterName).orElse(null);
+    }
+
+    @Override
+    public EvaluatorState getEvaluatorState() {
+        return evaluatorState;
+    }
+}
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardExpressionLanguageCompiler.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardExpressionLanguageCompiler.java
index 01a4eb9..fe95d87 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardExpressionLanguageCompiler.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardExpressionLanguageCompiler.java
@@ -18,26 +18,25 @@ package org.apache.nifi.attribute.expression.language;
 
 import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException;
 import org.apache.nifi.expression.AttributeExpression;
-import org.apache.nifi.expression.ExpressionLanguageCompiler;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
+import org.apache.nifi.expression.ExpressionLanguageCompiler;
+import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.registry.VariableRegistry;
 
 public class StandardExpressionLanguageCompiler implements ExpressionLanguageCompiler {
 
     private final VariableRegistry variableRegistry;
+    private final ParameterLookup parameterLookup;
 
-    public StandardExpressionLanguageCompiler() {
-        this.variableRegistry = VariableRegistry.EMPTY_REGISTRY;
-    }
-
-    public StandardExpressionLanguageCompiler(final VariableRegistry variableRegistry) {
+    public StandardExpressionLanguageCompiler(final VariableRegistry variableRegistry, final ParameterLookup parameterLookup) {
         this.variableRegistry = variableRegistry;
+        this.parameterLookup = parameterLookup;
     }
 
     @Override
     public AttributeExpression compile(final String expression) throws IllegalArgumentException {
         try {
-            return new StandardAttributeExpression(Query.compile(expression),variableRegistry);
+            return new StandardAttributeExpression(Query.compile(expression), variableRegistry, parameterLookup);
         } catch (final AttributeExpressionLanguageParsingException e) {
             throw new IllegalArgumentException(e.getMessage());
         }
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 b9fa720..545be0f 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,7 +16,6 @@
  */
 package org.apache.nifi.attribute.expression.language;
 
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
 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;
@@ -32,7 +31,6 @@ import org.apache.nifi.processor.exception.ProcessException;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 
 public class StandardPreparedQuery implements PreparedQuery {
@@ -46,19 +44,19 @@ public class StandardPreparedQuery implements PreparedQuery {
     }
 
     @Override
-    public String evaluateExpressions(final Map<String, String> valMap, final AttributeValueDecorator decorator, final Map<String, String> stateVariables) throws ProcessException {
+    public String evaluateExpressions(final EvaluationContext evaluationContext, final AttributeValueDecorator decorator) throws ProcessException {
         if (expressions.isEmpty()) {
             return EMPTY_STRING;
         }
         if (expressions.size() == 1) {
-            final String evaluated = expressions.get(0).evaluate(valMap, decorator, stateVariables);
+            final String evaluated = expressions.get(0).evaluate(evaluationContext, decorator);
             return evaluated == null ? EMPTY_STRING : evaluated;
         }
 
         final StringBuilder sb = new StringBuilder();
 
         for (final Expression expression : expressions) {
-            final String evaluated = expression.evaluate(valMap, decorator, stateVariables);
+            final String evaluated = expression.evaluate(evaluationContext, decorator);
 
             if (evaluated != null) {
                 sb.append(evaluated);
@@ -68,11 +66,6 @@ public class StandardPreparedQuery implements PreparedQuery {
         return sb.toString();
     }
 
-    @Override
-    public String evaluateExpressions(final Map<String, String> valMap, final AttributeValueDecorator decorator)
-            throws ProcessException {
-        return evaluateExpressions(valMap, decorator, null);
-    }
 
     @Override
     public boolean isExpressionLanguagePresent() {
@@ -100,7 +93,7 @@ public class StandardPreparedQuery implements PreparedQuery {
                     final Evaluator<String> nameEval = attributeEval.getNameEvaluator();
 
                     if (nameEval instanceof StringLiteralEvaluator) {
-                        final String referencedVar = nameEval.evaluate(Collections.emptyMap(), new EvaluatorState()).getValue();
+                        final String referencedVar = nameEval.evaluate(new StandardEvaluationContext(Collections.emptyMap())).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/StandardPropertyValue.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPropertyValue.java
index f6b6c70..4ffce2b 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPropertyValue.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPropertyValue.java
@@ -16,32 +16,34 @@
  */
 package org.apache.nifi.attribute.expression.language;
 
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
 import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.ControllerServiceLookup;
 import org.apache.nifi.expression.AttributeValueDecorator;
 import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.registry.VariableRegistry;
 import org.apache.nifi.util.FormatUtils;
 
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
 public class StandardPropertyValue implements PropertyValue {
 
     private final String rawValue;
     private final ControllerServiceLookup serviceLookup;
     private final PreparedQuery preparedQuery;
     private final VariableRegistry variableRegistry;
+    private final ParameterLookup parameterLookup;
 
-    public StandardPropertyValue(final String rawValue, final ControllerServiceLookup serviceLookup) {
-        this(rawValue, serviceLookup, Query.prepare(rawValue), VariableRegistry.EMPTY_REGISTRY);
+    public StandardPropertyValue(final String rawValue, final ControllerServiceLookup serviceLookup, final ParameterLookup parameterLookup) {
+        this(rawValue, serviceLookup, parameterLookup, Query.prepare(rawValue), VariableRegistry.EMPTY_REGISTRY);
     }
 
-    public StandardPropertyValue(final String rawValue, final ControllerServiceLookup serviceLookup, final VariableRegistry variableRegistry) {
-        this(rawValue, serviceLookup, Query.prepare(rawValue), variableRegistry);
+    public StandardPropertyValue(final String rawValue, final ControllerServiceLookup serviceLookup, final ParameterLookup parameterLookup, final VariableRegistry variableRegistry) {
+        this(rawValue, serviceLookup, parameterLookup, Query.prepare(rawValue), variableRegistry);
     }
 
     /**
@@ -49,7 +51,7 @@ public class StandardPropertyValue implements PropertyValue {
      * lookup and indicates whether or not the rawValue contains any NiFi
      * Expressions. If it is unknown whether or not the value contains any NiFi
      * Expressions, the
-     * {@link #StandardPropertyValue(String, ControllerServiceLookup, VariableRegistry)}
+     * {@link #StandardPropertyValue(String, ControllerServiceLookup, ParameterLookup, VariableRegistry)}
      * constructor should be used or <code>true</code> should be passed.
      * However, if it is known that the value contains no NiFi Expression, that
      * information should be provided so that calls to
@@ -57,15 +59,17 @@ public class StandardPropertyValue implements PropertyValue {
      *
      * @param rawValue value
      * @param serviceLookup lookup
+     * @param  parameterLookup the parameter lookup
      * @param preparedQuery query
      * @param variableRegistry variableRegistry
      */
-    public StandardPropertyValue(final String rawValue, final ControllerServiceLookup serviceLookup, final PreparedQuery preparedQuery,
+    public StandardPropertyValue(final String rawValue, final ControllerServiceLookup serviceLookup, final ParameterLookup parameterLookup, final PreparedQuery preparedQuery,
             final VariableRegistry variableRegistry) {
         this.rawValue = rawValue;
         this.serviceLookup = serviceLookup;
         this.preparedQuery = preparedQuery;
         this.variableRegistry = variableRegistry;
+        this.parameterLookup = parameterLookup == null ? ParameterLookup.EMPTY : parameterLookup;
     }
 
     @Override
@@ -157,8 +161,10 @@ public class StandardPropertyValue implements PropertyValue {
         }
 
         final ValueLookup lookup = new ValueLookup(variableRegistry, flowFile, additionalAttributes);
-        final String evaluated = preparedQuery.evaluateExpressions(lookup, decorator, stateValues);
-        return new StandardPropertyValue(evaluated, serviceLookup, new EmptyPreparedQuery(evaluated), null);
+        final EvaluationContext evaluationContext = new StandardEvaluationContext(lookup, stateValues, parameterLookup);
+        final String evaluated = preparedQuery.evaluateExpressions(evaluationContext, decorator);
+
+        return new StandardPropertyValue(evaluated, serviceLookup, parameterLookup, new EmptyPreparedQuery(evaluated), null);
     }
 
     @Override
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java
index 74dafce..d6a87d5 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java
@@ -17,8 +17,6 @@
 
 package org.apache.nifi.attribute.expression.language;
 
-import java.util.Map;
-
 import org.apache.nifi.expression.AttributeValueDecorator;
 
 public class StringLiteralExpression implements Expression {
@@ -29,7 +27,7 @@ public class StringLiteralExpression implements Expression {
     }
 
     @Override
-    public String evaluate(Map<String, String> variables, AttributeValueDecorator decorator, Map<String, String> stateVariables) {
+    public String evaluate(final EvaluationContext evaluationContext, AttributeValueDecorator decorator) {
         return value;
     }
 }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/ValueLookup.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/ValueLookup.java
index 06c1877..29225e7 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/ValueLookup.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/ValueLookup.java
@@ -16,6 +16,10 @@
  */
 package org.apache.nifi.attribute.expression.language;
 
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.registry.VariableDescriptor;
+import org.apache.nifi.registry.VariableRegistry;
+
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -24,9 +28,6 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
 
 /**
  * A convenience class to encapsulate the logic of variable substitution
@@ -35,8 +36,8 @@ import org.apache.nifi.registry.VariableRegistry;
  */
 final class ValueLookup implements Map<String, String> {
 
-    final List<Map<String, String>> maps = new ArrayList<>();
-    final VariableRegistry registry;
+    private final List<Map<String, String>> maps = new ArrayList<>();
+    private final VariableRegistry registry;
 
     /**
      * Constructs a ValueLookup where values are looked up first based any
@@ -56,6 +57,7 @@ final class ValueLookup implements Map<String, String> {
                 maps.add(map);
             }
         }
+
         if (flowFile != null) {
             maps.add(ValueLookup.extractFlowFileProperties(flowFile));
             maps.add(flowFile.getAttributes());
@@ -146,6 +148,14 @@ final class ValueLookup implements Map<String, String> {
         throw new UnsupportedOperationException();
     }
 
+    public Set<String> getKeysAddressableByMultiMatch() {
+        final Set<String> keys = new HashSet<>();
+        for (final Map<String, String> map : maps) {
+            keys.addAll(map.keySet());
+        }
+        return keys;
+    }
+
     @Override
     public Set<String> keySet() {
         final Set<String> keySet = new HashSet<>();
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 749c3be..eb258a4 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
@@ -24,11 +24,11 @@ import org.antlr.runtime.tree.Tree;
 import org.apache.nifi.attribute.expression.language.CompiledExpression;
 import org.apache.nifi.attribute.expression.language.Query;
 import org.apache.nifi.attribute.expression.language.Query.Range;
+import org.apache.nifi.attribute.expression.language.StandardEvaluationContext;
 import org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionLexer;
 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.EvaluatorState;
 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;
@@ -118,6 +118,7 @@ import org.apache.nifi.attribute.expression.language.evaluation.selection.Mappin
 import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiAttributeEvaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiMatchAttributeEvaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiNamedAttributeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.ParameterEvaluator;
 import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageException;
 import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
@@ -125,6 +126,7 @@ import org.apache.nifi.flowfile.FlowFile;
 
 import java.net.UnknownHostException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -186,6 +188,7 @@ import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpre
 import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NOT_NULL;
 import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NOW;
 import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.OR;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.PARAMETER_REFERENCE;
 import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.PLUS;
 import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.PREPEND;
 import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.RANDOM;
@@ -299,7 +302,7 @@ public class ExpressionCompiler {
         }
 
         final Evaluator<?> rootEvaluator = getRootSubjectEvaluator(evaluator);
-        if (rootEvaluator != null && rootEvaluator instanceof MultiAttributeEvaluator) {
+        if (rootEvaluator instanceof MultiAttributeEvaluator) {
             final MultiAttributeEvaluator multiAttrEval = (MultiAttributeEvaluator) rootEvaluator;
             switch (multiAttrEval.getEvaluationType()) {
                 case ALL_ATTRIBUTES:
@@ -925,6 +928,12 @@ public class ExpressionCompiler {
             case EXPRESSION: {
                 return buildExpressionEvaluator(tree);
             }
+            case PARAMETER_REFERENCE: {
+                final String parameterName = tree.getChild(0).getText();
+                final ParameterEvaluator parameterEvaluator = new ParameterEvaluator(parameterName);
+                evaluators.add(parameterEvaluator);
+                return parameterEvaluator;
+            }
             case ATTRIBUTE_REFERENCE: {
                 final Evaluator<?> childEvaluator = buildEvaluator(tree.getChild(0));
                 if (childEvaluator instanceof MultiAttributeEvaluator) {
@@ -948,7 +957,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, new EvaluatorState()).getValue());
+                    attributeNames.add(newStringLiteralEvaluator(tree.getChild(i).getText()).evaluate(new StandardEvaluationContext(Collections.emptyMap())).getValue());
                 }
 
                 switch (multiAttrType) {
@@ -1099,7 +1108,8 @@ public class ExpressionCompiler {
 
         final List<Range> ranges = Query.extractExpressionRanges(literalValue);
         if (ranges.isEmpty()) {
-            return addToken(new StringLiteralEvaluator(literalValue), literalValue);
+            final List<Range> escapedRanges = Query.extractEscapedRanges(literalValue);
+            return newStringLiteralEvaluatorForEscapedRanges(literalValue, escapedRanges);
         }
 
         final List<Evaluator<?>> evaluators = new ArrayList<>();
@@ -1142,6 +1152,44 @@ public class ExpressionCompiler {
         return lastEvaluator;
     }
 
+    private Evaluator<String> newStringLiteralEvaluatorForEscapedRanges(final String literalValue, final List<Range> escapedRanges) {
+        if (escapedRanges.isEmpty()) {
+            return addToken(new StringLiteralEvaluator(literalValue), literalValue);
+        }
+
+        int lastIndex = 0;
+        final List<Evaluator<?>> evaluators = new ArrayList<>();
+        for (final Range range : escapedRanges) {
+            final String treeText = literalValue.substring(range.getStart(), range.getEnd() + 1);
+
+            if (range.getStart() > lastIndex) {
+                evaluators.add(new StringLiteralEvaluator(literalValue.substring(lastIndex, range.getStart())));
+            }
+
+            final Evaluator<?> evaluator = new StringLiteralEvaluator(Query.unescape(treeText));
+            evaluators.add(evaluator);
+            lastIndex = range.getEnd() + 1;
+        }
+
+
+        final Range lastRange = escapedRanges.get(escapedRanges.size() - 1);
+        if (lastRange.getEnd() + 1 < literalValue.length()) {
+            final String treeText = literalValue.substring(lastRange.getEnd() + 1);
+            evaluators.add(new StringLiteralEvaluator(treeText));
+        }
+
+        if (evaluators.size() == 1) {
+            return toStringEvaluator(evaluators.get(0));
+        }
+
+        Evaluator<String> lastEvaluator = toStringEvaluator(evaluators.get(0));
+        for (int i = 1; i < evaluators.size(); i++) {
+            lastEvaluator = new AppendEvaluator(lastEvaluator, toStringEvaluator(evaluators.get(i)));
+        }
+
+        this.evaluators.addAll(evaluators);
+        return lastEvaluator;
+    }
 
     private Evaluator<Boolean> buildBooleanEvaluator(final Tree tree) {
         switch (tree.getType()) {
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 5e63d16..c75427f 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
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation;
 
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
 
 public abstract class BooleanEvaluator implements Evaluator<Boolean> {
@@ -27,7 +28,7 @@ public abstract class BooleanEvaluator implements Evaluator<Boolean> {
     }
 
     @Override
-    public int getEvaluationsRemaining(final EvaluatorState context) {
+    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 352952e..4e48025 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
@@ -16,10 +16,11 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation;
 
-import java.util.Date;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
 
+import java.util.Date;
+
 public abstract class DateEvaluator implements Evaluator<Date> {
     private String token;
 
@@ -29,7 +30,7 @@ public abstract class DateEvaluator implements Evaluator<Date> {
     }
 
     @Override
-    public int getEvaluationsRemaining(final EvaluatorState context) {
+    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/DateQueryResult.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateQueryResult.java
index a77bbe9..3d55e2a 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateQueryResult.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateQueryResult.java
@@ -16,10 +16,10 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation;
 
-import java.util.Date;
-
 import org.apache.nifi.expression.AttributeExpression.ResultType;
 
+import java.util.Date;
+
 public class DateQueryResult implements QueryResult<Date> {
 
     private final Date date;
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 7602690..1134fb6 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
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation;
 
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
 
 public abstract class DecimalEvaluator implements Evaluator<Double> {
@@ -27,7 +28,7 @@ public abstract class DecimalEvaluator implements Evaluator<Double> {
     }
 
     @Override
-    public int getEvaluationsRemaining(final EvaluatorState context) {
+    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/Evaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/Evaluator.java
index 499f93d..332f7b8 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
@@ -16,17 +16,16 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation;
 
-import java.util.Map;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
 
 public interface Evaluator<T> {
 
-    QueryResult<T> evaluate(Map<String, String> attributes, EvaluatorState context);
+    QueryResult<T> evaluate(EvaluationContext evaluationContext);
 
     ResultType getResultType();
 
-    int getEvaluationsRemaining(EvaluatorState context);
+    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 99d054f..d1f931a 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
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation;
 
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
 
 public abstract class NumberEvaluator implements Evaluator<Number> {
@@ -27,7 +28,7 @@ public abstract class NumberEvaluator implements Evaluator<Number> {
     }
 
     @Override
-    public int getEvaluationsRemaining(final EvaluatorState context) {
+    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 e3cdad4..2948a84 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
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation;
 
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
 
 public abstract class StringEvaluator implements Evaluator<String> {
@@ -27,7 +28,7 @@ public abstract class StringEvaluator implements Evaluator<String> {
     }
 
     @Override
-    public int getEvaluationsRemaining(EvaluatorState context) {
+    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 1389dbc..2cab771 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
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation;
 
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
 
 public abstract class WholeNumberEvaluator implements Evaluator<Long> {
@@ -27,7 +28,7 @@ public abstract class WholeNumberEvaluator implements Evaluator<Long> {
     }
 
     @Override
-    public int getEvaluationsRemaining(final EvaluatorState context) {
+    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 7d0928c..7371dff 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
@@ -16,11 +16,9 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.cast;
 
-import java.util.Map;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 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 +32,8 @@ public class BooleanCastEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final QueryResult<String> result = subjectEvaluator.evaluate(attributes, context);
+    public QueryResult<Boolean> evaluate(final EvaluationContext evaluationContext) {
+        final QueryResult<String> result = subjectEvaluator.evaluate(evaluationContext);
         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 33c82a1..b7c4428 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
@@ -16,17 +16,9 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.cast;
 
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.Locale;
-import java.util.Map;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 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;
@@ -35,6 +27,13 @@ import org.apache.nifi.attribute.expression.language.exception.AttributeExpressi
 import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
 
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Locale;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
 public class DateCastEvaluator extends DateEvaluator {
 
     public static final String DATE_TO_STRING_FORMAT = "EEE MMM dd HH:mm:ss zzz yyyy";
@@ -57,8 +56,8 @@ public class DateCastEvaluator extends DateEvaluator {
     }
 
     @Override
-    public QueryResult<Date> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final QueryResult<?> result = subjectEvaluator.evaluate(attributes, context);
+    public QueryResult<Date> evaluate(final EvaluationContext evaluationContext) {
+        final QueryResult<?> result = subjectEvaluator.evaluate(evaluationContext);
         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 5cbfde2..6f84bc6 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
@@ -16,10 +16,10 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.cast;
 
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 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;
@@ -29,8 +29,6 @@ import org.apache.nifi.attribute.expression.language.evaluation.util.NumberParsi
 import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
 
-import java.util.Map;
-
 public class DecimalCastEvaluator extends DecimalEvaluator {
 
     private final Evaluator<?> subjectEvaluator;
@@ -43,8 +41,8 @@ public class DecimalCastEvaluator extends DecimalEvaluator {
     }
 
     @Override
-    public QueryResult<Double> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final QueryResult<?> result = subjectEvaluator.evaluate(attributes, context);
+    public QueryResult<Double> evaluate(final EvaluationContext evaluationContext) {
+        final QueryResult<?> result = subjectEvaluator.evaluate(evaluationContext);
         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 c9bda44..5de82a1 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
@@ -16,9 +16,9 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.cast;
 
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 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;
@@ -29,8 +29,6 @@ import org.apache.nifi.attribute.expression.language.evaluation.util.NumberParsi
 import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
 
-import java.util.Map;
-
 public class NumberCastEvaluator extends NumberEvaluator {
 
     private final Evaluator<?> subjectEvaluator;
@@ -43,8 +41,8 @@ public class NumberCastEvaluator extends NumberEvaluator {
     }
 
     @Override
-    public QueryResult<Number> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final QueryResult<?> result = subjectEvaluator.evaluate(attributes, context);
+    public QueryResult<Number> evaluate(final EvaluationContext evaluationContext) {
+        final QueryResult<?> result = subjectEvaluator.evaluate(evaluationContext);
         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 37ff485..ff36b52 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.cast;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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 +31,8 @@ public class StringCastEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final QueryResult<?> result = subjectEvaluator.evaluate(attributes, context);
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final QueryResult<?> result = subjectEvaluator.evaluate(evaluationContext);
         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 4dfefea..d74ff18 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
@@ -16,11 +16,9 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.cast;
 
-import java.util.Map;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 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;
@@ -43,8 +41,8 @@ public class WholeNumberCastEvaluator extends WholeNumberEvaluator {
     }
 
     @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final QueryResult<?> result = subjectEvaluator.evaluate(attributes, context);
+    public QueryResult<Long> evaluate(final EvaluationContext evaluationContext) {
+        final QueryResult<?> result = subjectEvaluator.evaluate(evaluationContext);
         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 db4f000..ae326da 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
@@ -16,11 +16,9 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -35,8 +33,8 @@ public class AndEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final QueryResult<Boolean> subjectValue = subjectEvaluator.evaluate(attributes, context);
+    public QueryResult<Boolean> evaluate(final EvaluationContext evaluationContext) {
+        final QueryResult<Boolean> subjectValue = subjectEvaluator.evaluate(evaluationContext);
         if (subjectValue == null) {
             return new BooleanQueryResult(null);
         }
@@ -48,19 +46,19 @@ 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);
+        final BooleanQueryResult rhsResult = evaluationContext.getEvaluatorState().getState(this, BooleanQueryResult.class);
         if (rhsResult != null) {
             return rhsResult;
         }
 
-        final QueryResult<Boolean> rhsValue = rhsEvaluator.evaluate(attributes, context);
+        final QueryResult<Boolean> rhsValue = rhsEvaluator.evaluate(evaluationContext);
         BooleanQueryResult result;
         if (rhsValue == null) {
             result = new BooleanQueryResult(false);
         } else {
             result = new BooleanQueryResult(rhsValue.getValue());
         }
-        context.putState(this, result);
+        evaluationContext.getEvaluatorState().putState(this, result);
         return result;
     }
 
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 5541398..cdb7408 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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,9 +33,9 @@ public class AppendEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
-        final String appendValue = appendEvaluator.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).getValue();
+        final String appendValue = appendEvaluator.evaluate(evaluationContext).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 39147c2..fdf360d 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
@@ -16,16 +16,15 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.io.UnsupportedEncodingException;
-import java.util.Base64;
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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.io.UnsupportedEncodingException;
+import java.util.Base64;
+
 public class Base64DecodeEvaluator extends StringEvaluator {
 
     private final Evaluator<String> subject;
@@ -35,8 +34,8 @@ public class Base64DecodeEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).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 7fef980..4ed4ecb 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
@@ -16,16 +16,15 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.io.UnsupportedEncodingException;
-import java.util.Base64;
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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.nio.charset.StandardCharsets;
+import java.util.Base64;
+
 public class Base64EncodeEvaluator extends StringEvaluator {
 
     private final Evaluator<String> subject;
@@ -35,17 +34,13 @@ public class Base64EncodeEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new StringQueryResult(null);
         }
 
-        try {
-            return new StringQueryResult(Base64.getEncoder().encodeToString(subjectValue.getBytes("UTF-8")));
-        } catch (final UnsupportedEncodingException e) {
-            return null;    // won't happen.
-        }
+        return new StringQueryResult(Base64.getEncoder().encodeToString(subjectValue.getBytes(StandardCharsets.UTF_8)));
     }
 
     @Override
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 fbd3e8a..3ca0526 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
@@ -16,11 +16,9 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-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.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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;
@@ -76,8 +74,8 @@ public class CharSequenceTranslatorEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).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 4de3dd4..df4fd29 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
@@ -16,11 +16,9 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -35,13 +33,13 @@ public class ContainsEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<Boolean> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new BooleanQueryResult(false);
         }
 
-        final String searchString = search.evaluate(attributes, context).getValue();
+        final String searchString = search.evaluate(evaluationContext).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 396c30c..5ee5cec 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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;
@@ -35,13 +33,13 @@ public class DivideEvaluator extends NumberEvaluator {
     }
 
     @Override
-    public QueryResult<Number> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final Number subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<Number> evaluate(final EvaluationContext evaluationContext) {
+        final Number subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new NumberQueryResult(null);
         }
 
-        final Number divide = divideValue.evaluate(attributes, context).getValue();
+        final Number divide = divideValue.evaluate(evaluationContext).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 c40cb1d..3d915bb 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
@@ -16,11 +16,9 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -35,13 +33,13 @@ public class EndsWithEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<Boolean> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new BooleanQueryResult(false);
         }
 
-        final String searchString = search.evaluate(attributes, context).getValue();
+        final String searchString = search.evaluate(evaluationContext).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 a00a96a..597bfd7 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
@@ -16,16 +16,15 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Date;
-import java.util.Map;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 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;
 
+import java.util.Date;
+
 public class EqualsEvaluator extends BooleanEvaluator {
 
     private final Evaluator<?> subject;
@@ -37,13 +36,13 @@ public class EqualsEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final Object a = subject.evaluate(attributes, context).getValue();
+    public QueryResult<Boolean> evaluate(final EvaluationContext evaluationContext) {
+        final Object a = subject.evaluate(evaluationContext).getValue();
         if (a == null) {
             return new BooleanQueryResult(false);
         }
 
-        final Object b = compareTo.evaluate(attributes, context).getValue();
+        final Object b = compareTo.evaluate(evaluationContext).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 b2364dd..af6a6e1 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
@@ -16,11 +16,9 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -35,13 +33,13 @@ public class EqualsIgnoreCaseEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final Object a = subject.evaluate(attributes, context).getValue();
+    public QueryResult<Boolean> evaluate(final EvaluationContext evaluationContext) {
+        final Object a = subject.evaluate(evaluationContext).getValue();
         if (a == null) {
             return new BooleanQueryResult(false);
         }
 
-        final Object b = compareTo.evaluate(attributes, context).getValue();
+        final Object b = compareTo.evaluate(evaluationContext).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 7b10153..2193b8c 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
@@ -16,16 +16,17 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-import java.util.regex.Pattern;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
+import org.apache.nifi.attribute.expression.language.StandardEvaluationContext;
 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.EvaluatorState;
 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;
 
+import java.util.Collections;
+import java.util.regex.Pattern;
+
 public class FindEvaluator extends BooleanEvaluator {
 
     private final Evaluator<String> subject;
@@ -40,21 +41,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, new EvaluatorState()).getValue());
+            this.compiledPattern = Pattern.compile(search.evaluate(new StandardEvaluationContext(Collections.emptyMap())).getValue());
         } else {
             this.compiledPattern = null;
         }
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<Boolean> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new BooleanQueryResult(false);
         }
         final Pattern pattern;
         if (compiledPattern == null) {
-            String expression = search.evaluate(attributes, context).getValue();
+            String expression = search.evaluate(evaluationContext).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 53540bb..1daa427 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
@@ -16,19 +16,18 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.Locale;
-import java.util.Map;
-import java.util.TimeZone;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.DateEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
 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.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Locale;
+import java.util.TimeZone;
+
 public class FormatEvaluator extends StringEvaluator {
 
     private final DateEvaluator subject;
@@ -42,13 +41,13 @@ public class FormatEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final Date subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final Date subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new StringQueryResult(null);
         }
 
-        final QueryResult<String> formatResult = format.evaluate(attributes, context);
+        final QueryResult<String> formatResult = format.evaluate(evaluationContext);
         final String format = formatResult.getValue();
         if (format == null) {
             return null;
@@ -57,7 +56,7 @@ public class FormatEvaluator extends StringEvaluator {
         final SimpleDateFormat sdf = new SimpleDateFormat(format, Locale.US);
 
         if(timeZone != null) {
-            final QueryResult<String> tzResult = timeZone.evaluate(attributes, context);
+            final QueryResult<String> tzResult = timeZone.evaluate(evaluationContext);
             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 2470b5b..47c31e2 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,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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;
@@ -35,13 +33,13 @@ public class FromRadixEvaluator extends WholeNumberEvaluator {
     }
 
     @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String result = numberEvaluator.evaluate(attributes, context).getValue();
+    public QueryResult<Long> evaluate(final EvaluationContext evaluationContext) {
+        final String result = numberEvaluator.evaluate(evaluationContext).getValue();
         if (result == null) {
             return new WholeNumberQueryResult(null);
         }
 
-        final Long radix = radixEvaluator.evaluate(attributes, context).getValue();
+        final Long radix = radixEvaluator.evaluate(evaluationContext).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 cc715de..f6144b8 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
@@ -17,9 +17,7 @@
 
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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,13 +63,13 @@ public class GetDelimitedFieldEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subject = subjectEval.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final String subject = subjectEval.evaluate(evaluationContext).getValue();
         if (subject == null || subject.isEmpty()) {
             return new StringQueryResult("");
         }
 
-        final Long index = indexEval.evaluate(attributes, context).getValue();
+        final Long index = indexEval.evaluate(evaluationContext).getValue();
         if (index == null) {
             throw new AttributeExpressionLanguageException("Cannot evaluate getDelimitedField function because the index (which field to obtain) was not specified");
         }
@@ -79,7 +77,7 @@ public class GetDelimitedFieldEvaluator extends StringEvaluator {
             return new StringQueryResult("");
         }
 
-        final String delimiter = delimiterEval.evaluate(attributes, context).getValue();
+        final String delimiter = delimiterEval.evaluate(evaluationContext).getValue();
         if (delimiter == null || delimiter.isEmpty()) {
             throw new AttributeExpressionLanguageException("Cannot evaluate getDelimitedField function because the delimiter was not specified");
         } else if (delimiter.length() > 1) {
@@ -87,7 +85,7 @@ public class GetDelimitedFieldEvaluator extends StringEvaluator {
                 + "\", but only a single character is allowed.");
         }
 
-        final String quoteString = quoteCharEval.evaluate(attributes, context).getValue();
+        final String quoteString = quoteCharEval.evaluate(evaluationContext).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");
@@ -96,7 +94,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, context).getValue();
+        final String escapeString = escapeCharEval.evaluate(evaluationContext).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");
@@ -105,7 +103,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, context).getValue();
+        Boolean stripChars = stripCharsEval.evaluate(evaluationContext).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 8bf7111..8f3b0d0 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,10 +17,7 @@
 
 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.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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,21 +32,13 @@ public class GetStateVariableEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(Map<String, String> attributes, final EvaluatorState context) {
-        if (!(attributes instanceof AttributesAndState)){
-            return new StringQueryResult(null);
-        }
-
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new StringQueryResult(null);
         }
 
-        AttributesAndState attributesAndState = (AttributesAndState) attributes;
-
-        Map<String, String> stateMap = attributesAndState.getStateMap();
-        String stateValue = stateMap.get(subjectValue);
-
+        final String stateValue = evaluationContext.getState(subjectValue);
         return new StringQueryResult(stateValue);
     }
 
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 51c14f1..265a98a 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
@@ -16,11 +16,9 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -35,13 +33,13 @@ public class GreaterThanEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final Number subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<Boolean> evaluate(final EvaluationContext evaluationContext) {
+        final Number subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new BooleanQueryResult(false);
         }
 
-        final Number comparisonValue = comparison.evaluate(attributes, context).getValue();
+        final Number comparisonValue = comparison.evaluate(evaluationContext).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 fcb4db5..05b4ae7 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
@@ -16,11 +16,9 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -35,13 +33,13 @@ public class GreaterThanOrEqualEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final Number subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<Boolean> evaluate(final EvaluationContext evaluationContext) {
+        final Number subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new BooleanQueryResult(false);
         }
 
-        final Number comparisonValue = comparison.evaluate(attributes, context).getValue();
+        final Number comparisonValue = comparison.evaluate(evaluationContext).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 c0c1e5a..f7d5176 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
@@ -16,16 +16,16 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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.net.InetAddress;
+import java.net.UnknownHostException;
+
+
 public class HostnameEvaluator extends StringEvaluator {
 
     private final StringQueryResult hostname;
@@ -47,7 +47,7 @@ public class HostnameEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
         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 dcde526..f6e1249 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
@@ -16,16 +16,15 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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.net.InetAddress;
+import java.net.UnknownHostException;
+
 public class IPEvaluator extends StringEvaluator {
 
     private final StringQueryResult ipAddress;
@@ -35,7 +34,7 @@ public class IPEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
         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 b1e21a9..9eee405 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,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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;
@@ -37,12 +35,12 @@ public class IfElseEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final QueryResult<Boolean> subjectValue = subject.evaluate(attributes, context);
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final QueryResult<Boolean> subjectValue = subject.evaluate(evaluationContext);
         if (subjectValue == null) {
             return new StringQueryResult(null);
         }
-        final String ifElseValue = (Boolean.TRUE.equals(subjectValue.getValue())) ? trueEvaluator.evaluate(attributes, context).getValue() : falseEvaluator.evaluate(attributes, context).getValue();
+        final String ifElseValue = (Boolean.TRUE.equals(subjectValue.getValue())) ? trueEvaluator.evaluate(evaluationContext).getValue() : falseEvaluator.evaluate(evaluationContext).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 8fc81e4..a62fa70 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
@@ -16,15 +16,14 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.List;
-import java.util.Map;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
+import java.util.List;
+
 public class InEvaluator extends BooleanEvaluator {
 
     private final Evaluator<String> subject;
@@ -36,15 +35,15 @@ public class InEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<Boolean> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new BooleanQueryResult(false);
         }
 
         boolean isInList = false;
         for (Evaluator<String> evaluator : search) {
-            final String searchString = evaluator.evaluate(attributes, context).getValue();
+            final String searchString = evaluator.evaluate(evaluationContext).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 7979afb..229d096 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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;
@@ -35,13 +33,12 @@ public class IndexOfEvaluator extends WholeNumberEvaluator {
     }
 
     @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<Long> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new WholeNumberQueryResult(-1L);
         }
-        final String indexEvalValue = indexEvaluator.evaluate(attributes, context).getValue();
-
+        final String indexEvalValue = indexEvaluator.evaluate(evaluationContext).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 9253a03..228b7f5 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
@@ -16,11 +16,9 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -33,8 +31,8 @@ public class IsEmptyEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final Object subjectValue = subjectEvaluator.evaluate(attributes, context).getValue();
+    public QueryResult<Boolean> evaluate(final EvaluationContext evaluationContext) {
+        final Object subjectValue = subjectEvaluator.evaluate(evaluationContext).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 22f16bc..6955b32 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
@@ -16,11 +16,9 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -33,8 +31,8 @@ public class IsNullEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final Object subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<Boolean> evaluate(final EvaluationContext evaluationContext) {
+        final Object subjectValue = subject.evaluate(evaluationContext).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 a085af4..aadb324 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
@@ -16,11 +16,14 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import com.jayway.jsonpath.Configuration;
+import com.jayway.jsonpath.DocumentContext;
+import com.jayway.jsonpath.InvalidJsonException;
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.json.JacksonJsonProvider;
+import com.jayway.jsonpath.spi.json.JsonProvider;
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
+import org.apache.nifi.attribute.expression.language.StandardEvaluationContext;
 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,12 +31,10 @@ import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResul
 import org.apache.nifi.attribute.expression.language.evaluation.literals.StringLiteralEvaluator;
 import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageException;
 
-import com.jayway.jsonpath.Configuration;
-import com.jayway.jsonpath.DocumentContext;
-import com.jayway.jsonpath.InvalidJsonException;
-import com.jayway.jsonpath.JsonPath;
-import com.jayway.jsonpath.spi.json.JacksonJsonProvider;
-import com.jayway.jsonpath.spi.json.JsonProvider;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
 
 
 public class JsonPathEvaluator extends StringEvaluator {
@@ -53,7 +54,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, new EvaluatorState()).getValue());
+            precompiledJsonPathExp = compileJsonPathExpression(jsonPathExp.evaluate(new StandardEvaluationContext(Collections.emptyMap())).getValue());
         } else {
             precompiledJsonPathExp = null;
         }
@@ -61,8 +62,8 @@ public class JsonPathEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null || subjectValue.length() == 0) {
             throw new  AttributeExpressionLanguageException("Subject is empty");
         }
@@ -77,7 +78,7 @@ public class JsonPathEvaluator extends StringEvaluator {
         if (precompiledJsonPathExp != null) {
             compiledJsonPath = precompiledJsonPathExp;
         } else {
-            compiledJsonPath = compileJsonPathExpression(jsonPathExp.evaluate(attributes, context).getValue());
+            compiledJsonPath = compileJsonPathExpression(jsonPathExp.evaluate(evaluationContext).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 fe311b0..8209c4a 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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;
@@ -35,12 +33,12 @@ public class LastIndexOfEvaluator extends WholeNumberEvaluator {
     }
 
     @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<Long> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new WholeNumberQueryResult(-1L);
         }
-        final String indexEvalValue = indexEvaluator.evaluate(attributes, context).getValue();
+        final String indexEvalValue = indexEvaluator.evaluate(evaluationContext).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 8e4d2a8..d0f513b 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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;
@@ -33,8 +31,8 @@ public class LengthEvaluator extends WholeNumberEvaluator {
     }
 
     @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<Long> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).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 59a014e..fd123d1 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
@@ -16,11 +16,9 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -35,13 +33,13 @@ public class LessThanEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final Number subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<Boolean> evaluate(final EvaluationContext evaluationContext) {
+        final Number subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new BooleanQueryResult(false);
         }
 
-        final Number comparisonValue = comparison.evaluate(attributes, context).getValue();
+        final Number comparisonValue = comparison.evaluate(evaluationContext).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 3ce14bd..6ffba46 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
@@ -16,11 +16,9 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -35,13 +33,13 @@ public class LessThanOrEqualEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final Number subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<Boolean> evaluate(final EvaluationContext evaluationContext) {
+        final Number subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new BooleanQueryResult(false);
         }
 
-        final Number comparisonValue = comparison.evaluate(attributes, context).getValue();
+        final Number comparisonValue = comparison.evaluate(evaluationContext).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 b970905..5aac0de 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
@@ -16,16 +16,17 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-import java.util.regex.Pattern;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
+import org.apache.nifi.attribute.expression.language.StandardEvaluationContext;
 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.EvaluatorState;
 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;
 
+import java.util.Collections;
+import java.util.regex.Pattern;
+
 public class MatchesEvaluator extends BooleanEvaluator {
 
     private final Evaluator<String> subject;
@@ -40,21 +41,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, new EvaluatorState()).getValue());
+            this.compiledPattern = Pattern.compile(search.evaluate(new StandardEvaluationContext(Collections.emptyMap())).getValue());
         } else {
             this.compiledPattern = null;
         }
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<Boolean> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new BooleanQueryResult(false);
         }
         final Pattern pattern;
         if (compiledPattern == null) {
-            String expression = search.evaluate(attributes, context).getValue();
+            String expression = search.evaluate(evaluationContext).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 96bc3a6..709a8eb 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,17 +16,16 @@
  */
 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.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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;
+
 public class MathEvaluator extends NumberEvaluator {
 
     private final Evaluator<Number> subject;
@@ -40,15 +39,15 @@ public class MathEvaluator extends NumberEvaluator {
     }
 
     @Override
-    public QueryResult<Number> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String methodNamedValue = methodName.evaluate(attributes, context).getValue();
+    public QueryResult<Number> evaluate(final EvaluationContext evaluationContext) {
+        final String methodNamedValue = methodName.evaluate(evaluationContext).getValue();
         if (methodNamedValue == null) {
             return new NumberQueryResult(null);
         }
 
         final Number subjectValue;
         if(subject != null) {
-            subjectValue = subject.evaluate(attributes, context).getValue();
+            subjectValue = subject.evaluate(evaluationContext).getValue();
             if(subjectValue == null){
                 return new NumberQueryResult(null);
             }
@@ -58,7 +57,7 @@ public class MathEvaluator extends NumberEvaluator {
 
         final Number optionalArgValue;
         if(optionalArg != null) {
-            optionalArgValue = optionalArg.evaluate(attributes, context).getValue();
+            optionalArgValue = optionalArg.evaluate(evaluationContext).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 7011496..0095104 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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;
@@ -35,13 +33,13 @@ public class MinusEvaluator extends NumberEvaluator {
     }
 
     @Override
-    public QueryResult<Number> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final Number subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<Number> evaluate(final EvaluationContext evaluationContext) {
+        final Number subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new NumberQueryResult(null);
         }
 
-        final Number minus = minusValue.evaluate(attributes, context).getValue();
+        final Number minus = minusValue.evaluate(evaluationContext).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 3652327..b36ec23 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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;
@@ -35,13 +33,13 @@ public class ModEvaluator extends NumberEvaluator {
     }
 
     @Override
-    public QueryResult<Number> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final Number subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<Number> evaluate(final EvaluationContext evaluationContext) {
+        final Number subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new NumberQueryResult(null);
         }
 
-        final Number mod = modValue.evaluate(attributes, context).getValue();
+        final Number mod = modValue.evaluate(evaluationContext).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 9ee42f1..a704ec0 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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;
@@ -35,13 +33,13 @@ public class MultiplyEvaluator extends NumberEvaluator {
     }
 
     @Override
-    public QueryResult<Number> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final Number subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<Number> evaluate(final EvaluationContext evaluationContext) {
+        final Number subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new NumberQueryResult(null);
         }
 
-        final Number multiply = multiplyValue.evaluate(attributes, context).getValue();
+        final Number multiply = multiplyValue.evaluate(evaluationContext).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 05cbcdc..cb47ccc 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
@@ -16,11 +16,9 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -33,8 +31,8 @@ public class NotEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final QueryResult<Boolean> subjectValue = subjectEvaluator.evaluate(attributes, context);
+    public QueryResult<Boolean> evaluate(final EvaluationContext evaluationContext) {
+        final QueryResult<Boolean> subjectValue = subjectEvaluator.evaluate(evaluationContext);
         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 47823b1..cefc21b 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
@@ -16,11 +16,9 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -33,8 +31,8 @@ public class NotNullEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final Object subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<Boolean> evaluate(final EvaluationContext evaluationContext) {
+        final Object subjectValue = subject.evaluate(evaluationContext).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 a8bfb12..43d84cc 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
@@ -16,19 +16,18 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Date;
-import java.util.Map;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
+import java.util.Date;
+
 public class NowEvaluator extends DateEvaluator {
 
     @Override
-    public QueryResult<Date> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
+    public QueryResult<Date> evaluate(final EvaluationContext evaluationContext) {
         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 b06b0cc..731a225 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
@@ -16,15 +16,14 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Date;
-import java.util.Map;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
+import java.util.Date;
+
 public class NumberToDateEvaluator extends DateEvaluator {
 
     private final Evaluator<Long> subject;
@@ -34,8 +33,8 @@ public class NumberToDateEvaluator extends DateEvaluator {
     }
 
     @Override
-    public QueryResult<Date> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final QueryResult<Long> result = subject.evaluate(attributes, context);
+    public QueryResult<Date> evaluate(final EvaluationContext evaluationContext) {
+        final QueryResult<Long> result = subject.evaluate(evaluationContext);
         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 d364bfc..62bde1a 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
@@ -16,21 +16,20 @@
  */
 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.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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.concurrent.atomic.AtomicLong;
+
 public class OneUpSequenceEvaluator extends WholeNumberEvaluator {
 
     private static final AtomicLong value = new AtomicLong(0L);
 
     @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
+    public QueryResult<Long> evaluate(final EvaluationContext evaluationContext) {
         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 71f3417..f9769f9 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
@@ -16,11 +16,9 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -35,8 +33,8 @@ public class OrEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final QueryResult<Boolean> subjectValue = subjectEvaluator.evaluate(attributes, context);
+    public QueryResult<Boolean> evaluate(final EvaluationContext evaluationContext) {
+        final QueryResult<Boolean> subjectValue = subjectEvaluator.evaluate(evaluationContext);
         if (subjectValue == null) {
             return new BooleanQueryResult(null);
         }
@@ -48,19 +46,19 @@ 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);
+        final BooleanQueryResult rhsResult = evaluationContext.getEvaluatorState().getState(this, BooleanQueryResult.class);
         if (rhsResult != null) {
             return rhsResult;
         }
 
-        final QueryResult<Boolean> rhsValue = rhsEvaluator.evaluate(attributes, context);
+        final QueryResult<Boolean> rhsValue = rhsEvaluator.evaluate(evaluationContext);
         BooleanQueryResult result;
         if (rhsValue == null) {
             result = new BooleanQueryResult(false);
         } else {
             result = new BooleanQueryResult(rhsValue.getValue());
         }
-        context.putState(this, result);
+        evaluationContext.getEvaluatorState().putState(this, result);
         return result;
     }
 
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 8649289..c9ca323 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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;
@@ -35,13 +33,13 @@ public class PlusEvaluator extends NumberEvaluator {
     }
 
     @Override
-    public QueryResult<Number> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final Number subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<Number> evaluate(final EvaluationContext evaluationContext) {
+        final Number subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new NumberQueryResult(null);
         }
 
-        final Number plus = plusValue.evaluate(attributes, context).getValue();
+        final Number plus = plusValue.evaluate(evaluationContext).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 6a6fdd1..0dcc05e 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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,9 +33,9 @@ public class PrependEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
-        final String prependValue = prependEvaluator.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).getValue();
+        final String prependValue = prependEvaluator.evaluate(evaluationContext).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 d68d491..739bccd 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
@@ -16,22 +16,21 @@
  */
 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.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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.Random;
+
 public class RandomNumberGeneratorEvaluator extends WholeNumberEvaluator {
 
     private static final Random RNG = new Random();
 
 
     @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
+    public QueryResult<Long> evaluate(final EvaluationContext evaluationContext) {
         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 5c9c85f..aa0caaf 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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;
@@ -37,13 +35,13 @@ public class ReplaceAllEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new StringQueryResult(null);
         }
-        final String searchValue = search.evaluate(attributes, context).getValue();
-        final String replacementValue = replacement.evaluate(attributes, context).getValue();
+        final String searchValue = search.evaluate(evaluationContext).getValue();
+        final String replacementValue = replacement.evaluate(evaluationContext).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 9528aa5..4157cc4 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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 +32,12 @@ public class ReplaceEmptyEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final QueryResult<String> subjectResult = subjectEvaluator.evaluate(attributes, context);
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final QueryResult<String> subjectResult = subjectEvaluator.evaluate(evaluationContext);
         final String subjectValue = subjectResult.getValue();
         final boolean isEmpty = subjectValue == null || subjectValue.toString().trim().isEmpty();
         if (isEmpty) {
-            return replacementEvaluator.evaluate(attributes, context);
+            return replacementEvaluator.evaluate(evaluationContext);
         } 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 b95fc50..00b7038 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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;
@@ -37,13 +35,13 @@ public class ReplaceEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new StringQueryResult(null);
         }
-        final String searchValue = search.evaluate(attributes, context).getValue();
-        final String replacementValue = replacement.evaluate(attributes, context).getValue();
+        final String searchValue = search.evaluate(evaluationContext).getValue();
+        final String replacementValue = replacement.evaluate(evaluationContext).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 490c32c..10eaeab 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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;
@@ -37,13 +35,13 @@ public class ReplaceFirstEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new StringQueryResult(null);
         }
-        final String searchValue = search.evaluate(attributes, context).getValue();
-        final String replacementValue = replacement.evaluate(attributes, context).getValue();
+        final String searchValue = search.evaluate(evaluationContext).getValue();
+        final String replacementValue = replacement.evaluate(evaluationContext).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 f044454..d9c76a0 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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,9 +33,9 @@ public class ReplaceNullEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
-        return new StringQueryResult(subjectValue == null ? resultEvaluator.evaluate(attributes, context).getValue() : subjectValue);
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).getValue();
+        return new StringQueryResult(subjectValue == null ? resultEvaluator.evaluate(evaluationContext).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 f17047a..8c6e394 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
@@ -16,11 +16,9 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -35,13 +33,13 @@ public class StartsWithEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<Boolean> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new BooleanQueryResult(false);
         }
 
-        final String searchString = search.evaluate(attributes, context).getValue();
+        final String searchString = search.evaluate(evaluationContext).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 e87fb17..f20c084 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
@@ -16,20 +16,19 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.Locale;
-import java.util.Map;
-import java.util.TimeZone;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 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;
 
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Locale;
+import java.util.TimeZone;
+
 public class StringToDateEvaluator extends DateEvaluator {
 
     private final Evaluator<String> subject;
@@ -43,9 +42,9 @@ public class StringToDateEvaluator extends DateEvaluator {
     }
 
     @Override
-    public QueryResult<Date> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
-        final String formatValue = format.evaluate(attributes, context).getValue();
+    public QueryResult<Date> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).getValue();
+        final String formatValue = format.evaluate(evaluationContext).getValue();
         if (subjectValue == null || formatValue == null) {
             return new DateQueryResult(null);
         }
@@ -53,7 +52,7 @@ public class StringToDateEvaluator extends DateEvaluator {
         final SimpleDateFormat sdf = new SimpleDateFormat(formatValue, Locale.US);
 
         if(timeZone != null) {
-            final QueryResult<String> tzResult = timeZone.evaluate(attributes, context);
+            final QueryResult<String> tzResult = timeZone.evaluate(evaluationContext);
             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 9c51d12..f4f24c0 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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,12 +33,12 @@ public class SubstringAfterEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new StringQueryResult("");
         }
-        final String afterValue = afterEvaluator.evaluate(attributes, context).getValue();
+        final String afterValue = afterEvaluator.evaluate(evaluationContext).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 37dce3d..ba282bd 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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,12 +33,12 @@ public class SubstringAfterLastEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new StringQueryResult("");
         }
-        final String afterValue = afterEvaluator.evaluate(attributes, context).getValue();
+        final String afterValue = afterEvaluator.evaluate(evaluationContext).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 2b8e8b8..2b69bc3 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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,12 +33,12 @@ public class SubstringBeforeEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new StringQueryResult("");
         }
-        final String beforeValue = beforeEvaluator.evaluate(attributes, context).getValue();
+        final String beforeValue = beforeEvaluator.evaluate(evaluationContext).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 664fc37..87e0109 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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,12 +33,12 @@ public class SubstringBeforeLastEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new StringQueryResult("");
         }
-        final String beforeValue = beforeEvaluator.evaluate(attributes, context).getValue();
+        final String beforeValue = beforeEvaluator.evaluate(evaluationContext).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 a617c3b..23990fb 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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;
@@ -43,16 +41,16 @@ public class SubstringEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).getValue();
         if (subjectValue == null) {
             return new StringQueryResult("");
         }
-        final int startIndexValue = startIndex.evaluate(attributes, context).getValue().intValue();
+        final int startIndexValue = startIndex.evaluate(evaluationContext).getValue().intValue();
         if (endIndex == null) {
             return new StringQueryResult(subjectValue.substring(startIndexValue));
         } else {
-            final int endIndexValue = endIndex.evaluate(attributes, context).getValue().intValue();
+            final int endIndexValue = endIndex.evaluate(evaluationContext).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 3413309..8e1cd87 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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 +25,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, final EvaluatorState context) {
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
         // 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 6941f46..13043f4 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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 +31,8 @@ public class ToLowerEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).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 3a0db49..d9afb16 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
@@ -16,15 +16,14 @@
  */
 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.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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.Arrays;
+
 public class ToRadixEvaluator extends StringEvaluator {
 
     private final Evaluator<Long> numberEvaluator;
@@ -42,20 +41,20 @@ public class ToRadixEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final Long result = numberEvaluator.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final Long result = numberEvaluator.evaluate(evaluationContext).getValue();
         if (result == null) {
             return new StringQueryResult(null);
         }
 
-        final Long radix = radixEvaluator.evaluate(attributes, context).getValue();
+        final Long radix = radixEvaluator.evaluate(evaluationContext).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, context).getValue();
+            final Long minimumWidth = minimumWidthEvaluator.evaluate(evaluationContext).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 bc8f5a8..b80fe5b 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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 +31,8 @@ public class ToStringEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final Object result = subject.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final Object result = subject.evaluate(evaluationContext).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 342ab2b..2870623 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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 +31,8 @@ public class ToUpperEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).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 ad96892..436019f 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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 +31,8 @@ public class TrimEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).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 61156c4..c422318 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
@@ -16,16 +16,15 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.io.UnsupportedEncodingException;
-import java.net.URLDecoder;
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+
 public class UrlDecodeEvaluator extends StringEvaluator {
 
     private final Evaluator<String> subject;
@@ -35,8 +34,8 @@ public class UrlDecodeEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).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 ceed25d..7ca8812 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
@@ -16,16 +16,15 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import java.io.UnsupportedEncodingException;
-import java.net.URLEncoder;
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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.io.UnsupportedEncodingException;
+import java.net.URLEncoder;
+
 public class UrlEncodeEvaluator extends StringEvaluator {
 
     private final Evaluator<String> subject;
@@ -35,8 +34,8 @@ public class UrlEncodeEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String subjectValue = subject.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final String subjectValue = subject.evaluate(evaluationContext).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 bbe5866..e2dbf23 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
@@ -16,19 +16,18 @@
  */
 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.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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.UUID;
+
 public class UuidEvaluator extends StringEvaluator {
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
         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 cd04eec..d144e3a 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
@@ -16,11 +16,9 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.literals;
 
-import java.util.Map;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -33,7 +31,7 @@ public class BooleanLiteralEvaluator extends BooleanEvaluator {
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
+    public QueryResult<Boolean> evaluate(final EvaluationContext evaluationContext) {
         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 cf351eb..f9dc34f 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
@@ -16,14 +16,12 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.literals;
 
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
-import java.util.Map;
-
 
 public class DecimalLiteralEvaluator extends DecimalEvaluator {
 
@@ -34,7 +32,7 @@ public class DecimalLiteralEvaluator extends DecimalEvaluator {
     }
 
     @Override
-    public QueryResult<Double> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
+    public QueryResult<Double> evaluate(final EvaluationContext evaluationContext) {
         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 9c0c0ab..410449b 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.literals;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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;
@@ -66,7 +64,7 @@ public class StringLiteralEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
         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 f44591a..7f430eb 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.literals;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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 +30,8 @@ public class ToLiteralEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final Object result = argEvaluator.evaluate(attributes, context);
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final Object result = argEvaluator.evaluate(evaluationContext);
         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 f476ba0..c22a484 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.literals;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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;
@@ -33,7 +31,7 @@ public class WholeNumberLiteralEvaluator extends WholeNumberEvaluator {
     }
 
     @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
+    public QueryResult<Long> evaluate(final EvaluationContext evaluationContext) {
         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 59df3b9..37a7451 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.reduce;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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,9 +32,9 @@ public class CountEvaluator extends WholeNumberEvaluator implements ReduceEvalua
     }
 
     @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final QueryResult<?> result = subjectEvaluator.evaluate(attributes, context);
-        Long count = context.getState(this, Long.class);
+    public QueryResult<Long> evaluate(final EvaluationContext evaluationContext) {
+        final QueryResult<?> result = subjectEvaluator.evaluate(evaluationContext);
+        Long count = evaluationContext.getEvaluatorState().getState(this, Long.class);
         if (count == null) {
             count = 0L;
         }
@@ -49,7 +47,7 @@ public class CountEvaluator extends WholeNumberEvaluator implements ReduceEvalua
         }
 
         count++;
-        context.putState(this, count);
+        evaluationContext.getEvaluatorState().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 8b7cdf4..62d249e 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.reduce;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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,17 +33,17 @@ public class JoinEvaluator extends StringEvaluator implements ReduceEvaluator<St
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        String subject = subjectEvaluator.evaluate(attributes, context).getValue();
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        String subject = subjectEvaluator.evaluate(evaluationContext).getValue();
         if (subject == null) {
             subject = "";
         }
 
-        final String delimiter = delimiterEvaluator.evaluate(attributes, context).getValue();
-        State state = context.getState(this, State.class);
+        final String delimiter = delimiterEvaluator.evaluate(evaluationContext).getValue();
+        State state = evaluationContext.getEvaluatorState().getState(this, State.class);
         if (state == null) {
             state = new State();
-            context.putState(this, state);
+            evaluationContext.getEvaluatorState().putState(this, state);
         }
         if (state.evalCount > 0) {
             state.sb.append(delimiter);
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 c2a7144..6b4331b 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
@@ -16,11 +16,9 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.selection;
 
-import java.util.Map;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -35,8 +33,8 @@ public class AllAttributesEvaluator extends BooleanEvaluator implements Iteratin
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        QueryResult<Boolean> attributeValueQuery = booleanEvaluator.evaluate(attributes, context);
+    public QueryResult<Boolean> evaluate(final EvaluationContext evaluationContext) {
+        QueryResult<Boolean> attributeValueQuery = booleanEvaluator.evaluate(evaluationContext);
         Boolean result = attributeValueQuery.getValue();
         if (result == null) {
             return new BooleanQueryResult(false);
@@ -46,8 +44,8 @@ public class AllAttributesEvaluator extends BooleanEvaluator implements Iteratin
             return new BooleanQueryResult(false);
         }
 
-        while (multiAttributeEvaluator.getEvaluationsRemaining(context) > 0) {
-            attributeValueQuery = booleanEvaluator.evaluate(attributes, context);
+        while (multiAttributeEvaluator.getEvaluationsRemaining(evaluationContext) > 0) {
+            attributeValueQuery = booleanEvaluator.evaluate(evaluationContext);
             result = attributeValueQuery.getValue();
             if (result != null && !result) {
                 return attributeValueQuery;
@@ -58,7 +56,7 @@ public class AllAttributesEvaluator extends BooleanEvaluator implements Iteratin
     }
 
     @Override
-    public int getEvaluationsRemaining(final EvaluatorState context) {
+    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 a800b45..1fd8fad 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
@@ -16,11 +16,9 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.selection;
 
-import java.util.Map;
-
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 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.EvaluatorState;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
@@ -35,8 +33,8 @@ public class AnyAttributeEvaluator extends BooleanEvaluator implements Iterating
     }
 
     @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        QueryResult<Boolean> attributeValueQuery = booleanEvaluator.evaluate(attributes, context);
+    public QueryResult<Boolean> evaluate(final EvaluationContext evaluationContext) {
+        QueryResult<Boolean> attributeValueQuery = booleanEvaluator.evaluate(evaluationContext);
         Boolean result = attributeValueQuery.getValue();
         if (result == null) {
             return new BooleanQueryResult(false);
@@ -46,8 +44,8 @@ public class AnyAttributeEvaluator extends BooleanEvaluator implements Iterating
             return new BooleanQueryResult(true);
         }
 
-        while (multiAttributeEvaluator.getEvaluationsRemaining(context) > 0) {
-            attributeValueQuery = booleanEvaluator.evaluate(attributes, context);
+        while (multiAttributeEvaluator.getEvaluationsRemaining(evaluationContext) > 0) {
+            attributeValueQuery = booleanEvaluator.evaluate(evaluationContext);
             result = attributeValueQuery.getValue();
             if (result != null && result) {
                 return attributeValueQuery;
@@ -58,7 +56,7 @@ public class AnyAttributeEvaluator extends BooleanEvaluator implements Iterating
     }
 
     @Override
-    public int getEvaluationsRemaining(final EvaluatorState context) {
+    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 41c9853..ba89349 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.selection;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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,9 +31,9 @@ public class AttributeEvaluator extends StringEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        final String nameValue = nameEvaluator.evaluate(attributes, context).getValue();
-        final String attributeValue = attributes.get(nameValue);
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final String nameValue = nameEvaluator.evaluate(evaluationContext).getValue();
+        final String attributeValue = evaluationContext.getExpressionValue(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 5dd7b45..1858333 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.selection;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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;
@@ -36,20 +34,20 @@ public class DelineatedAttributeEvaluator extends MultiAttributeEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        State state = context.getState(this, State.class);
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        State state = evaluationContext.getEvaluatorState().getState(this, State.class);
         if (state == null) {
             state = new State();
-            context.putState(this, state);
+            evaluationContext.getEvaluatorState().putState(this, state);
         }
         if (state.delineatedValues == null) {
-            final QueryResult<String> subjectValue = subjectEvaluator.evaluate(attributes, context);
+            final QueryResult<String> subjectValue = subjectEvaluator.evaluate(evaluationContext);
             if (subjectValue.getValue() == null) {
                 state.evaluationsLeft = 0;
                 return new StringQueryResult(null);
             }
 
-            final QueryResult<String> delimiterValue = delimiterEvaluator.evaluate(attributes, context);
+            final QueryResult<String> delimiterValue = delimiterEvaluator.evaluate(evaluationContext);
             if (delimiterValue.getValue() == null) {
                 state.evaluationsLeft = 0;
                 return new StringQueryResult(null);
@@ -74,11 +72,11 @@ public class DelineatedAttributeEvaluator extends MultiAttributeEvaluator {
     }
 
     @Override
-    public int getEvaluationsRemaining(final EvaluatorState context) {
-        State state = context.getState(this, State.class);
+    public int getEvaluationsRemaining(final EvaluationContext evaluationContext) {
+        State state = evaluationContext.getEvaluatorState().getState(this, State.class);
         if (state == null) {
             state = new State();
-            context.putState(this, state);
+            evaluationContext.getEvaluatorState().putState(this, state);
         }
         return state.evaluationsLeft;
     }
@@ -93,7 +91,7 @@ public class DelineatedAttributeEvaluator extends MultiAttributeEvaluator {
         return evaluationType;
     }
 
-    private class State {
+    private static 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 9ea578d..632df63 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
@@ -16,9 +16,7 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.selection;
 
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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;
@@ -35,11 +33,11 @@ public class MappingEvaluator<T> implements Evaluator<T> {
     }
 
     @Override
-    public QueryResult<T> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        QueryResult<T> result = mappingEvaluator.evaluate(attributes, context);
+    public QueryResult<T> evaluate(final EvaluationContext evaluationContext) {
+        QueryResult<T> result = mappingEvaluator.evaluate(evaluationContext);
 
-        while (multiAttributeEvaluator.getEvaluationsRemaining(context) > 0) {
-            result = mappingEvaluator.evaluate(attributes, context);
+        while (multiAttributeEvaluator.getEvaluationsRemaining(evaluationContext) > 0) {
+            result = mappingEvaluator.evaluate(evaluationContext);
         }
 
         return result;
@@ -51,7 +49,7 @@ public class MappingEvaluator<T> implements Evaluator<T> {
     }
 
     @Override
-    public int getEvaluationsRemaining(final EvaluatorState context) {
+    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 9b3dcb9..d8da2a3 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
@@ -16,16 +16,15 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.selection;
 
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.regex.Pattern;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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;
 
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Pattern;
+
 public class MultiMatchAttributeEvaluator extends MultiAttributeEvaluator {
 
     private final List<Pattern> attributePatterns;
@@ -46,25 +45,25 @@ public class MultiMatchAttributeEvaluator extends MultiAttributeEvaluator {
      * @return number of remaining evaluations
      */
     @Override
-    public int getEvaluationsRemaining(final EvaluatorState context) {
-        State state = context.getState(this, State.class);
+    public int getEvaluationsRemaining(final EvaluationContext context) {
+        State state = context.getEvaluatorState().getState(this, State.class);
         if (state == null) {
             state = new State();
-            context.putState(this, state);
+            context.getEvaluatorState().putState(this, state);
         }
         return state.attributeNames.size() - state.evaluationCount;
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        State state = context.getState(this, State.class);
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        State state = evaluationContext.getEvaluatorState().getState(this, State.class);
         if (state == null) {
             state = new State();
-            context.putState(this, state);
+            evaluationContext.getEvaluatorState().putState(this, state);
         }
         if (state.evaluationCount == 0) {
             for (final Pattern pattern : attributePatterns) {
-                for (final String attrName : attributes.keySet()) {
+                for (final String attrName : evaluationContext.getExpressionKeys()) {
                     if (pattern.matcher(attrName).matches()) {
                         state.attributeNames.add(attrName);
                     }
@@ -76,7 +75,8 @@ public class MultiMatchAttributeEvaluator extends MultiAttributeEvaluator {
             return new StringQueryResult(null);
         }
 
-        return new StringQueryResult(attributes.get(state.attributeNames.get(state.evaluationCount++)));
+        final String attributeName = state.attributeNames.get(state.evaluationCount++);
+        return new StringQueryResult(evaluationContext.getExpressionValue(attributeName));
     }
 
     @Override
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 523376a..3658e5f 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
@@ -16,15 +16,15 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.selection;
 
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
 
+import java.util.ArrayList;
+import java.util.List;
+
 public class MultiNamedAttributeEvaluator extends MultiAttributeEvaluator {
 
     private final List<String> attributeNames;
@@ -36,11 +36,11 @@ public class MultiNamedAttributeEvaluator extends MultiAttributeEvaluator {
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        State state = context.getState(this, State.class);
+    public QueryResult<String> evaluate(EvaluationContext evaluationContext) {
+        State state = evaluationContext.getEvaluatorState().getState(this, State.class);
         if (state == null) {
             state = new State();
-            context.putState(this, state);
+            evaluationContext.getEvaluatorState().putState(this, state);
         }
         state.matchingAttributeNames = new ArrayList<>(attributeNames);
 
@@ -48,15 +48,16 @@ public class MultiNamedAttributeEvaluator extends MultiAttributeEvaluator {
             return new StringQueryResult(null);
         }
 
-        return new StringQueryResult(attributes.get(state.matchingAttributeNames.get(state.evaluationCount++)));
+        return new StringQueryResult(evaluationContext.getExpressionValue(state.matchingAttributeNames.get(state.evaluationCount++)));
     }
 
     @Override
-    public int getEvaluationsRemaining(final EvaluatorState context) {
-        State state = context.getState(this, State.class);
+    public int getEvaluationsRemaining(final EvaluationContext context) {
+        final EvaluatorState evaluatorState = context.getEvaluatorState();
+        State state = evaluatorState.getState(this, State.class);
         if (state == null) {
             state = new State();
-            context.putState(this, state);
+            evaluatorState.putState(this, state);
         }
         return state.matchingAttributeNames.size() - state.evaluationCount;
     }
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/selection/ParameterEvaluator.java
similarity index 69%
copy from nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IPEvaluator.java
copy to nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/ParameterEvaluator.java
index dcde526..a5f2585 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/selection/ParameterEvaluator.java
@@ -14,34 +14,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.attribute.expression.language.evaluation.functions;
+package org.apache.nifi.attribute.expression.language.evaluation.selection;
 
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.attribute.expression.language.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 org.apache.nifi.parameter.Parameter;
 
-public class IPEvaluator extends StringEvaluator {
-
-    private final StringQueryResult ipAddress;
+public class ParameterEvaluator extends StringEvaluator {
+    private final String parameterName;
 
-    public IPEvaluator() throws UnknownHostException {
-        ipAddress = new StringQueryResult(InetAddress.getLocalHost().getHostAddress());
+    public ParameterEvaluator(final String parameterName) {
+        this.parameterName = parameterName;
     }
 
     @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes, final EvaluatorState context) {
-        return ipAddress;
+    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
+        final Parameter parameter = evaluationContext.getParameter(parameterName);
+        return new StringQueryResult(parameter == null ? null : parameter.getValue());
     }
 
     @Override
     public Evaluator<?> getSubjectEvaluator() {
         return null;
     }
-
 }
diff --git a/nifi-commons/nifi-expression-language/src/test/groovy/org/apache/nifi/attribute/expression/language/QueryGroovyTest.groovy b/nifi-commons/nifi-expression-language/src/test/groovy/org/apache/nifi/attribute/expression/language/QueryGroovyTest.groovy
index bdd704d..78de467 100644
--- a/nifi-commons/nifi-expression-language/src/test/groovy/org/apache/nifi/attribute/expression/language/QueryGroovyTest.groovy
+++ b/nifi-commons/nifi-expression-language/src/test/groovy/org/apache/nifi/attribute/expression/language/QueryGroovyTest.groovy
@@ -75,10 +75,10 @@ public class QueryGroovyTest extends GroovyTestCase {
         Query replaceRepeatingQuery = Query.compile(REPLACE_REPEATING_EXPRESSION)
 
         // Act
-        QueryResult<?> replaceSingleResult = replaceSingleQuery.evaluate(attributes)
+        QueryResult<?> replaceSingleResult = replaceSingleQuery.evaluate(new StandardEvaluationContext(attributes))
         logger.info("Replace single result: ${replaceSingleResult.value}")
 
-        QueryResult<?> replaceRepeatingResult = replaceRepeatingQuery.evaluate(attributes)
+        QueryResult<?> replaceRepeatingResult = replaceRepeatingQuery.evaluate(new StandardEvaluationContext(attributes))
         logger.info("Replace repeating result: ${replaceRepeatingResult.value}")
 
         // Assert
@@ -115,10 +115,10 @@ public class QueryGroovyTest extends GroovyTestCase {
         Query replaceRepeatingQuery = Query.compile(REPLACE_REPEATING_EXPRESSION)
 
         // Act
-        QueryResult<?> replaceSingleResult = replaceSingleQuery.evaluate(attributes)
+        QueryResult<?> replaceSingleResult = replaceSingleQuery.evaluate(new StandardEvaluationContext(attributes))
         logger.info("Replace single result: ${replaceSingleResult.value}")
 
-        QueryResult<?> replaceRepeatingResult = replaceRepeatingQuery.evaluate(attributes)
+        QueryResult<?> replaceRepeatingResult = replaceRepeatingQuery.evaluate(new StandardEvaluationContext(attributes))
         logger.info("Replace repeating result: ${replaceRepeatingResult.value}")
 
         // Assert
@@ -155,10 +155,10 @@ public class QueryGroovyTest extends GroovyTestCase {
         Query replaceRepeatingQuery = Query.compile(REPLACE_REPEATING_EXPRESSION)
 
         // Act
-        QueryResult<?> replaceSingleResult = replaceSingleQuery.evaluate(attributes)
+        QueryResult<?> replaceSingleResult = replaceSingleQuery.evaluate(new StandardEvaluationContext(attributes))
         logger.info("Replace single result: ${replaceSingleResult.value}")
 
-        QueryResult<?> replaceRepeatingResult = replaceRepeatingQuery.evaluate(attributes)
+        QueryResult<?> replaceRepeatingResult = replaceRepeatingQuery.evaluate(new StandardEvaluationContext(attributes))
         logger.info("Replace repeating result: ${replaceRepeatingResult.value}")
 
         // Assert
@@ -187,7 +187,7 @@ public class QueryGroovyTest extends GroovyTestCase {
         final String REPLACE_ONLY_FIRST_PATTERN = /\w+\s\w+\b??/
 
         // Act
-        
+
         // Execute on both single and repeating with String#replace()
         String replaceSingleResult = attributes.single.replace(REPLACE_ONLY_FIRST_PATTERN, REPLACEMENT_VALUE)
         logger.info("Replace single result: ${replaceSingleResult}")
diff --git a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
index 1c679a4..fcd636a 100644
--- a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
+++ b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
@@ -24,6 +24,9 @@ import org.apache.nifi.attribute.expression.language.exception.AttributeExpressi
 import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
 import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterDescriptor;
+import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.registry.VariableRegistry;
 import org.junit.Assert;
 import org.junit.Ignore;
@@ -43,6 +46,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
+import java.util.Optional;
 
 import static java.lang.Double.NEGATIVE_INFINITY;
 import static java.lang.Double.NaN;
@@ -74,18 +78,19 @@ public class TestQuery {
         //System.out.println(Query.compile("").evaluate(null));
     }
 
+
     @Test
     public void testPrepareWithEscapeChar() {
         final Map<String, String> variables = Collections.singletonMap("foo", "bar");
 
-        assertEquals("bar${foo}$bar", Query.prepare("${foo}$${foo}$$${foo}").evaluateExpressions(variables, null));
+        assertEquals("bar${foo}$bar", Query.prepare("${foo}$${foo}$$${foo}").evaluateExpressions(new StandardEvaluationContext(variables), null));
 
         final PreparedQuery onlyEscapedQuery = Query.prepare("$${foo}");
-        final String onlyEscapedEvaluated = onlyEscapedQuery.evaluateExpressions(variables, null);
+        final String onlyEscapedEvaluated = onlyEscapedQuery.evaluateExpressions(new StandardEvaluationContext(variables), null);
         assertEquals("${foo}", onlyEscapedEvaluated);
 
         final PreparedQuery mixedQuery = Query.prepare("${foo}$${foo}");
-        final String mixedEvaluated = mixedQuery.evaluateExpressions(variables, null);
+        final String mixedEvaluated = mixedQuery.evaluateExpressions(new StandardEvaluationContext(variables), null);
         assertEquals("bar${foo}", mixedEvaluated);
     }
 
@@ -229,7 +234,7 @@ public class TestQuery {
         final Map<String, String> attributes = new HashMap<>();
         attributes.put("dateTime", "2013/11/18 10:22:27.678");
 
-        final QueryResult<?> result = query.evaluate(attributes);
+        final QueryResult<?> result = query.evaluate(new StandardEvaluationContext(attributes));
         assertEquals(ResultType.WHOLE_NUMBER, result.getResultType());
         assertEquals(1384788147678L, result.getValue());
     }
@@ -258,7 +263,7 @@ public class TestQuery {
         final Date roundedToNearestSecond = new Date(date.getTime() - millis);
         final String formatted = sdf.format(roundedToNearestSecond);
 
-        final QueryResult<?> result = query.evaluate(attributes);
+        final QueryResult<?> result = query.evaluate(new StandardEvaluationContext(attributes));
         assertEquals(ResultType.STRING, result.getResultType());
         assertEquals(formatted, result.getValue());
     }
@@ -279,6 +284,18 @@ public class TestQuery {
     }
 
     @Test
+    public void testParameterReference() {
+        final Map<String, String> attributes = Collections.emptyMap();
+        final Map<String, String> stateValues = Collections.emptyMap();
+        final Map<String, String> parameters = new HashMap<>();
+        parameters.put("test", "unit");
+
+        verifyEquals("${#{test}}", attributes, stateValues, parameters,"unit");
+        verifyEquals("${#{test}:append(' - '):append(#{test})}", attributes, stateValues, parameters,"unit - unit");
+    }
+
+
+    @Test
     public void testJsonPath() throws IOException {
         final Map<String, String> attributes = new HashMap<>();
         attributes.put("json", getResourceAsString("/json/address-book.json"));
@@ -398,7 +415,7 @@ public class TestQuery {
         Mockito.when(mockFlowFile.getLineageStartDate()).thenReturn(System.currentTimeMillis());
 
         final ValueLookup lookup = new ValueLookup(VariableRegistry.EMPTY_REGISTRY, mockFlowFile);
-        return Query.evaluateExpressions(queryString, lookup);
+        return Query.evaluateExpressions(queryString, lookup, ParameterLookup.EMPTY);
     }
 
     @Test
@@ -671,7 +688,7 @@ public class TestQuery {
         final String query = "${ abc:equals('abc'):or( \n\t${xx:isNull()}\n) }";
         assertEquals(ResultType.BOOLEAN, Query.getResultType(query));
         Query.validateExpression(query, false);
-        assertEquals("true", Query.evaluateExpressions(query, Collections.emptyMap()));
+        assertEquals("true", Query.evaluateExpressions(query, Collections.emptyMap(), ParameterLookup.EMPTY));
     }
 
     @Test
@@ -697,12 +714,12 @@ public class TestQuery {
                 + "}";
 
         Query query = Query.compile(expression);
-        QueryResult<?> result = query.evaluate(attributes);
+        QueryResult<?> result = query.evaluate(new StandardEvaluationContext(attributes));
         assertEquals(ResultType.STRING, result.getResultType());
         assertEquals("xyz", result.getValue());
 
         query = Query.compile("${abc:append('# hello') #good-bye \n}");
-        result = query.evaluate(attributes);
+        result = query.evaluate(new StandardEvaluationContext(attributes));
         assertEquals(ResultType.STRING, result.getResultType());
         assertEquals("xyz# hello", result.getValue());
     }
@@ -1602,7 +1619,7 @@ public class TestQuery {
         final List<String> expressions = Query.extractExpressions(query);
         assertEquals(1, expressions.size());
         assertEquals("${abc}", expressions.get(0));
-        assertEquals("{ xyz }", Query.evaluateExpressions(query, attributes));
+        assertEquals("{ xyz }", Query.evaluateExpressions(query, attributes, ParameterLookup.EMPTY));
     }
 
     @Test
@@ -1652,7 +1669,7 @@ public class TestQuery {
 
     QueryResult<?> getResult(String expr, Map<String, String> attrs) {
         final Query query = Query.compile(expr);
-        final QueryResult<?> result = query.evaluate(attrs);
+        final QueryResult<?> result = query.evaluate(new StandardEvaluationContext(attrs));
         return result;
     }
 
@@ -1823,15 +1840,26 @@ public class TestQuery {
     }
 
     private void verifyEquals(final String expression, final Map<String, String> attributes, final Object expectedResult) {
-        verifyEquals(expression,attributes, null, expectedResult);
+        verifyEquals(expression,attributes, null, ParameterLookup.EMPTY, expectedResult);
     }
 
     private void verifyEquals(final String expression, final Map<String, String> attributes, final Map<String, String> stateValues, final Object expectedResult) {
+        verifyEquals(expression, attributes, stateValues, ParameterLookup.EMPTY, expectedResult);
+    }
+
+    private void verifyEquals(final String expression, final Map<String, String> attributes, final Map<String, String> stateValues, final Map<String, String> parameters,
+                              final Object expectedResult) {
+
+        verifyEquals(expression, attributes, stateValues, new MapParameterLookup(parameters), expectedResult);
+    }
+
+    private void verifyEquals(final String expression, final Map<String, String> attributes, final Map<String, String> stateValues, final ParameterLookup parameterLookup,
+                              final Object expectedResult) {
         Query.validateExpression(expression, false);
-        assertEquals(String.valueOf(expectedResult), Query.evaluateExpressions(expression, attributes, null, stateValues));
+        assertEquals(String.valueOf(expectedResult), Query.evaluateExpressions(expression, attributes, null, stateValues, parameterLookup));
 
         final Query query = Query.compile(expression);
-        final QueryResult<?> result = query.evaluate(attributes, stateValues);
+        final QueryResult<?> result = query.evaluate(new StandardEvaluationContext(attributes, stateValues, parameterLookup));
 
         if (expectedResult instanceof Long) {
             if (ResultType.NUMBER.equals(result.getResultType())) {
@@ -1879,4 +1907,28 @@ public class TestQuery {
             return sb.toString();
         }
     }
+
+
+    private static class MapParameterLookup implements ParameterLookup {
+        private final Map<String, String> parameters;
+
+        public MapParameterLookup(final Map<String, String> parameters) {
+            this.parameters = parameters;
+        }
+
+        @Override
+        public Optional<Parameter> getParameter(final String parameterName) {
+            final String value = parameters.get(parameterName);
+            if (value == null) {
+                return Optional.empty();
+            }
+
+            return Optional.of(new Parameter(new ParameterDescriptor.Builder().name(parameterName).build(), value));
+        }
+
+        @Override
+        public boolean isEmpty() {
+            return parameters.isEmpty();
+        }
+    }
 }
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 556066f..d6767ba 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
@@ -16,9 +16,9 @@
  */
 package org.apache.nifi.attribute.expression.language;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import org.apache.nifi.parameter.ParameterLookup;
+import org.junit.Ignore;
+import org.junit.Test;
 
 import java.util.HashMap;
 import java.util.HashSet;
@@ -26,8 +26,9 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
-import org.junit.Ignore;
-import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 public class TestStandardPreparedQuery {
 
@@ -58,7 +59,7 @@ public class TestStandardPreparedQuery {
         final StandardPreparedQuery prepared = (StandardPreparedQuery) Query.prepare("${xx}");
         final long start = System.nanoTime();
         for (int i = 0; i < 10000000; i++) {
-            assertEquals("world", prepared.evaluateExpressions(attrs, null));
+            assertEquals("world", prepared.evaluateExpressions(new StandardEvaluationContext(attrs), null));
         }
         final long nanos = System.nanoTime() - start;
         System.out.println(TimeUnit.NANOSECONDS.toMillis(nanos));
@@ -72,7 +73,7 @@ public class TestStandardPreparedQuery {
 
         final long start = System.nanoTime();
         for (int i = 0; i < 10000000; i++) {
-            assertEquals("world", Query.evaluateExpressions("${xx}", attrs));
+            assertEquals("world", Query.evaluateExpressions("${xx}", attrs, ParameterLookup.EMPTY));
         }
         final long nanos = System.nanoTime() - start;
         System.out.println(TimeUnit.NANOSECONDS.toMillis(nanos));
@@ -95,12 +96,12 @@ public class TestStandardPreparedQuery {
         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));
+        assertEquals("false", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
+        assertEquals("false", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
         attributes.put("audience", "bla");
-        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+        assertEquals("false", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
         attributes.put("comma", "hello");
-        assertEquals("true", prepared.evaluateExpressions(attributes, null));
+        assertEquals("true", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
     }
 
     @Test
@@ -109,12 +110,12 @@ public class TestStandardPreparedQuery {
         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));
+        assertEquals("false", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
+        assertEquals("false", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
         attributes.put("comma", "hello");
-        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+        assertEquals("false", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
         attributes.put("question", "hello");
-        assertEquals("true", prepared.evaluateExpressions(attributes, null));
+        assertEquals("true", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
     }
 
     @Test
@@ -123,12 +124,12 @@ public class TestStandardPreparedQuery {
         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));
+        assertEquals("false", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
+        assertEquals("false", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
         attributes.put("audience", "bla");
-        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+        assertEquals("false", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
         attributes.put("auditorium", "hello");
-        assertEquals("true", prepared.evaluateExpressions(attributes, null));
+        assertEquals("true", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
     }
 
     @Test
@@ -137,12 +138,12 @@ public class TestStandardPreparedQuery {
         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));
+        assertEquals("true", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
+        assertEquals("true", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
         attributes.put("audience", "bla");
-        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+        assertEquals("false", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
         attributes.remove("audience");
-        assertEquals("true", prepared.evaluateExpressions(attributes, null));
+        assertEquals("true", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
     }
 
     @Test
@@ -150,12 +151,12 @@ public class TestStandardPreparedQuery {
         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));
+        assertEquals("true", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
+        assertEquals("true", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
         attributes.put("number_list", "1,2,3");
-        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+        assertEquals("false", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
         attributes.put("number_list", "5");
-        assertEquals("true", prepared.evaluateExpressions(attributes, null));
+        assertEquals("true", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
     }
 
     @Test
@@ -163,12 +164,12 @@ public class TestStandardPreparedQuery {
         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));
+        assertEquals("true", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
+        assertEquals("true", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
         attributes.put("word_list", "beach,party,web");
-        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+        assertEquals("false", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
         attributes.put("word_list", "bee");
-        assertEquals("true", prepared.evaluateExpressions(attributes, null));
+        assertEquals("true", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
     }
 
     @Test
@@ -177,9 +178,9 @@ public class TestStandardPreparedQuery {
         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));
+        assertEquals("Hello World!", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
         attributes.put("boat", "Friend.");
-        assertEquals("Hello Friend.", prepared.evaluateExpressions(attributes, null));
+        assertEquals("Hello Friend.", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
     }
 
     @Test
@@ -188,9 +189,9 @@ public class TestStandardPreparedQuery {
         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));
+        assertEquals("1", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
         attributes.put("boat", "end");
-        assertEquals("2", prepared.evaluateExpressions(attributes, null));
+        assertEquals("2", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
     }
 
     @Test
@@ -199,9 +200,9 @@ public class TestStandardPreparedQuery {
         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));
+        assertEquals("true", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
         attributes.put("hello", "hi");
-        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+        assertEquals("false", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
     }
 
     @Test
@@ -209,9 +210,9 @@ public class TestStandardPreparedQuery {
         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));
+        assertEquals("true", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
         attributes.put("hello", "Hell");
-        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+        assertEquals("false", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
     }
 
     @Test
@@ -220,9 +221,9 @@ public class TestStandardPreparedQuery {
         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));
+        assertEquals("true", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
         attributes.put("hello", "hi");
-        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+        assertEquals("false", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
     }
 
     @Test
@@ -230,9 +231,9 @@ public class TestStandardPreparedQuery {
         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));
+        assertEquals("true", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
         attributes.put("hello", "Hell");
-        assertEquals("false", prepared.evaluateExpressions(attributes, null));
+        assertEquals("false", prepared.evaluateExpressions(new StandardEvaluationContext(attributes), null));
     }
 
     @Test
@@ -272,7 +273,7 @@ public class TestStandardPreparedQuery {
     }
 
     private String evaluate(final String query, final Map<String, String> attrs) {
-        final String evaluated = ((StandardPreparedQuery) Query.prepare(query)).evaluateExpressions(attrs, null);
+        final String evaluated = ((StandardPreparedQuery) Query.prepare(query)).evaluateExpressions(new StandardEvaluationContext(attrs), null);
         return evaluated;
     }
 
diff --git a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestValueLookup.java b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestValueLookup.java
index 1010ac7..97935fd 100644
--- a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestValueLookup.java
+++ b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestValueLookup.java
@@ -16,14 +16,16 @@
  */
 package org.apache.nifi.attribute.expression.language;
 
-import java.util.HashMap;
-import java.util.Map;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.registry.VariableRegistry;
 import org.junit.Test;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertFalse;
+
+import java.util.HashMap;
+import java.util.Map;
+
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 public class TestValueLookup {
 
diff --git a/nifi-commons/nifi-parameter/pom.xml b/nifi-commons/nifi-parameter/pom.xml
new file mode 100644
index 0000000..d37bfc0
--- /dev/null
+++ b/nifi-commons/nifi-parameter/pom.xml
@@ -0,0 +1,37 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+      http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+    <parent>
+        <artifactId>nifi-commons</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.10.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-parameter</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+        </dependency>
+    </dependencies>
+
+</project>
\ No newline at end of file
diff --git a/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/AbstractParameterParser.java b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/AbstractParameterParser.java
new file mode 100644
index 0000000..fd740e7
--- /dev/null
+++ b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/AbstractParameterParser.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.parameter;
+
+import java.util.List;
+
+public abstract class AbstractParameterParser implements ParameterParser {
+    protected static final char START_TAG = '#';
+    protected static final char OPEN_BRACE = '{';
+    protected static final char CLOSE_BRACE = '}';
+
+
+    protected ParameterToken parseParameterToken(final String input, final int startIndex, final int sequentialStartTags, final List<ParameterToken> tokens) {
+        int startCharIndex = startIndex - sequentialStartTags;
+        final int endCharIndex = input.indexOf(CLOSE_BRACE, startIndex);
+        if (endCharIndex < 0) {
+            return null;
+        }
+
+
+        final int numEscapedStartTags = (sequentialStartTags - 1)/2;
+        final int startOffset = startCharIndex + numEscapedStartTags * 2;
+        final String referenceText = input.substring(startOffset, endCharIndex + 1);
+
+        // If we have multiple escapes before the start tag, we need to add a StartCharacterEscape for each one.
+        // For example, if we have ###{foo}, then we should end up with a StartCharacterEscape followed by an actual Parameter Reference.
+        for (int escapes=0; escapes < numEscapedStartTags; escapes++) {
+            tokens.add(new StartCharacterEscape(startCharIndex + escapes * 2));
+        }
+
+        final ParameterToken token;
+        if (sequentialStartTags % 2 == 1) {
+            final String parameterName = input.substring(startCharIndex + sequentialStartTags + 1, endCharIndex);
+            token = new StandardParameterReference(parameterName, startOffset, endCharIndex, referenceText);
+        } else {
+            token = new EscapedParameterReference(startOffset, endCharIndex, referenceText);
+        }
+
+        tokens.add(token);
+        return token;
+    }
+}
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/EscapedParameterReference.java
similarity index 51%
copy from nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java
copy to nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/EscapedParameterReference.java
index e9ac03b..7417303 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java
+++ b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/EscapedParameterReference.java
@@ -14,39 +14,46 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.attribute.expression.language;
+package org.apache.nifi.parameter;
 
+public class EscapedParameterReference implements ParameterToken {
+    private final int startOffset;
+    private final int endOffset;
+    private final String referenceText;
 
-import java.util.Map;
-
-import org.apache.nifi.expression.AttributeValueDecorator;
-import org.apache.nifi.processor.exception.ProcessException;
-
-public class EmptyPreparedQuery implements PreparedQuery {
+    public EscapedParameterReference(final int startOffset, final int endOffset, final String referenceText) {
+        this.startOffset = startOffset;
+        this.endOffset = endOffset;
+        this.referenceText = referenceText;
+    }
 
-    private final String value;
+    @Override
+    public int getStartOffset() {
+        return startOffset;
+    }
 
-    EmptyPreparedQuery(final String value) {
-        this.value = value;
+    @Override
+    public int getEndOffset() {
+        return endOffset;
     }
 
     @Override
-    public String evaluateExpressions(Map<String, String> valueLookup, AttributeValueDecorator decorator) throws ProcessException {
-        return value;
+    public String getText() {
+        return referenceText;
     }
 
     @Override
-    public String evaluateExpressions(Map<String, String> attributes, AttributeValueDecorator decorator, Map<String, String> stateVariables) throws ProcessException {
-        return value;
+    public boolean isEscapeSequence() {
+        return true;
     }
 
     @Override
-    public boolean isExpressionLanguagePresent() {
+    public boolean isParameterReference() {
         return false;
     }
 
     @Override
-    public VariableImpact getVariableImpact() {
-        return VariableImpact.NEVER_IMPACTED;
+    public String getValue(final ParameterLookup parameterLookup) {
+        return referenceText.replace("##", "#");
     }
 }
diff --git a/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ExpressionLanguageAgnosticParameterParser.java b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ExpressionLanguageAgnosticParameterParser.java
new file mode 100644
index 0000000..2f28086
--- /dev/null
+++ b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ExpressionLanguageAgnosticParameterParser.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.parameter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class ExpressionLanguageAgnosticParameterParser extends AbstractParameterParser {
+    private static final Logger logger = LoggerFactory.getLogger(ExpressionLanguageAgnosticParameterParser.class);
+
+    @Override
+    public ParameterTokenList parseTokens(final String input) {
+        if (input == null || input.isEmpty()) {
+            return new StandardParameterTokenList(input, Collections.emptyList());
+        }
+
+        final List<ParameterToken> references = new ArrayList<>();
+        int sequentialStartTags = 0;
+
+        for (int i=0; i < input.length(); i++) {
+            final char c = input.charAt(i);
+
+            switch (c) {
+                case START_TAG:
+                    // If last character was a # character, then the previous character along with this character
+                    // represent an escaped literal # character. Otherwise, this character potentially represents
+                    // the start of a Parameter Reference.
+                    sequentialStartTags++;
+                    break;
+                case OPEN_BRACE:
+                    if (sequentialStartTags > 0) {
+                        final ParameterToken token = parseParameterToken(input, i, sequentialStartTags, references);
+
+                        // If we found a reference, skip 'i' to the end of the reference, since there can't be any other references before that point.
+                        if (token != null) {
+                            i = token.getEndOffset();
+                        }
+                    }
+
+                    break;
+                default:
+                    break;
+            }
+
+            if (c != START_TAG) {
+                sequentialStartTags = 0;
+            }
+        }
+
+        logger.debug("For input {} found {} Parameter references: {}", input, references.size(), references);
+        return new StandardParameterTokenList(input, references);
+    }
+}
diff --git a/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ExpressionLanguageAwareParameterParser.java b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ExpressionLanguageAwareParameterParser.java
new file mode 100644
index 0000000..79dddcf
--- /dev/null
+++ b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ExpressionLanguageAwareParameterParser.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.parameter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class ExpressionLanguageAwareParameterParser extends AbstractParameterParser implements ParameterParser {
+    private static final Logger logger = LoggerFactory.getLogger(ExpressionLanguageAwareParameterParser.class);
+    private static final char DOLLAR_SIGN = '$';
+
+
+    @Override
+    public ParameterTokenList parseTokens(final String input) {
+        if (input == null || input.isEmpty()) {
+            return new StandardParameterTokenList(input, Collections.emptyList());
+        }
+
+        final List<ParameterToken> tokens = new ArrayList<>();
+
+        int sequentialStartTags = 0;
+        boolean oddDollarCount = false;
+        char lastChar = 0;
+        int embeddedElCount = 0;
+        int expressionStart = -1;
+
+        for (int i=0; i < input.length(); i++) {
+            final char c = input.charAt(i);
+
+            switch (c) {
+                case START_TAG:
+                    // If last character was a # character, then the previous character along with this character
+                    // represent an escaped literal # character. Otherwise, this character potentially represents
+                    // the start of a Parameter Reference.
+                    sequentialStartTags++;
+                    break;
+                case OPEN_BRACE:
+                    if (oddDollarCount && lastChar == '$') {
+                        if (embeddedElCount == 0) {
+                            expressionStart = i - 1;
+                        }
+                    }
+
+                    // Keep track of the number of opening curly braces that we are embedded within,
+                    // if we are within an Expression. If we are outside of an Expression, we can just ignore
+                    // curly braces. This allows us to ignore the first character if the value is something
+                    // like: { ${abc} }
+                    // However, we will count the curly braces if we have something like: ${ $${abc} }
+                    if (expressionStart > -1) {
+                        embeddedElCount++;
+                        continue;
+                    }
+
+
+                    if (sequentialStartTags > 0) {
+                        final ParameterToken token = parseParameterToken(input, i, sequentialStartTags, tokens);
+
+                        // If we found a reference, skip 'i' to the end of the reference, since there can't be any other references before that point.
+                        if (token != null) {
+                            i = token.getEndOffset();
+                        }
+                    }
+
+                    break;
+                case CLOSE_BRACE:
+                    if (embeddedElCount <= 0) {
+                        continue;
+                    }
+
+                    if (--embeddedElCount == 0) {
+                        expressionStart = -1;
+                    }
+                    break;
+                case DOLLAR_SIGN:
+                    oddDollarCount = !oddDollarCount;
+                    break;
+                default:
+                    break;
+            }
+
+            if (c != START_TAG) {
+                sequentialStartTags = 0;
+            }
+
+            lastChar = c;
+        }
+
+        logger.debug("For input {} found {} Parameter references: {}", input, tokens.size(), tokens);
+        return new StandardParameterTokenList(input, tokens);
+    }
+
+}
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/Evaluator.java b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ParameterLookup.java
similarity index 51%
copy from nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/Evaluator.java
copy to nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ParameterLookup.java
index 499f93d..3d97170 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/Evaluator.java
+++ b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ParameterLookup.java
@@ -14,31 +14,35 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.attribute.expression.language.evaluation;
+package org.apache.nifi.parameter;
 
-import java.util.Map;
+import java.util.Optional;
 
-import org.apache.nifi.expression.AttributeExpression.ResultType;
-
-public interface Evaluator<T> {
-
-    QueryResult<T> evaluate(Map<String, String> attributes, EvaluatorState context);
-
-    ResultType getResultType();
-
-    int getEvaluationsRemaining(EvaluatorState context);
-
-    Evaluator<?> getSubjectEvaluator();
+public interface ParameterLookup {
 
     /**
-     * Sets the token that was used in the query to cause this Evaluator to be created
-     *
-     * @param token the token that caused this Evaluator to be created
+     * Returns the Parameter with the given name
+     * @param parameterName the name of the Parameter
+     * @return the Parameter with the given name or an empty Optional if no Parameter exists with that name
      */
-    void setToken(String token);
+    Optional<Parameter> getParameter(String parameterName);
 
     /**
-     * @return the token that caused this Evaluator to be created
+     * Returns false if any Parameters are available, true if no Parameters have been defined
+     * @return true if empty
      */
-    String getToken();
+    boolean isEmpty();
+
+
+    ParameterLookup EMPTY = new ParameterLookup() {
+        @Override
+        public Optional<Parameter> getParameter(final String parameterName) {
+            return Optional.empty();
+        }
+
+        @Override
+        public boolean isEmpty() {
+            return true;
+        }
+    };
 }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ParameterParser.java
similarity index 61%
copy from nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java
copy to nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ParameterParser.java
index 74dafce..e157353 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java
+++ b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ParameterParser.java
@@ -14,22 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.nifi.parameter;
 
-package org.apache.nifi.attribute.expression.language;
-
-import java.util.Map;
-
-import org.apache.nifi.expression.AttributeValueDecorator;
-
-public class StringLiteralExpression implements Expression {
-    private final String value;
-
-    public StringLiteralExpression(final String value) {
-        this.value = value;
-    }
-
-    @Override
-    public String evaluate(Map<String, String> variables, AttributeValueDecorator decorator, Map<String, String> stateVariables) {
-        return value;
-    }
+public interface ParameterParser {
+    /**
+     * Parses the given input, returning a ParameterTokenList that contains all Parameter tokens that were identified.
+     *
+     * @param input the input to parse
+     * @return a ParameterTokenList containing all parameter tokens identified
+     */
+    ParameterTokenList parseTokens(String input);
 }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ParameterReference.java
similarity index 61%
copy from nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java
copy to nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ParameterReference.java
index 74dafce..b8dd235 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java
+++ b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ParameterReference.java
@@ -14,22 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.nifi.parameter;
 
-package org.apache.nifi.attribute.expression.language;
-
-import java.util.Map;
-
-import org.apache.nifi.expression.AttributeValueDecorator;
-
-public class StringLiteralExpression implements Expression {
-    private final String value;
-
-    public StringLiteralExpression(final String value) {
-        this.value = value;
-    }
-
-    @Override
-    public String evaluate(Map<String, String> variables, AttributeValueDecorator decorator, Map<String, String> stateVariables) {
-        return value;
-    }
+public interface ParameterReference extends ParameterToken {
+    /**
+     * @return the name of the Parameter that is referenced
+     */
+    String getParameterName();
 }
diff --git a/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ParameterToken.java b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ParameterToken.java
new file mode 100644
index 0000000..f4e1b19
--- /dev/null
+++ b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ParameterToken.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.parameter;
+
+/**
+ * A token encountered when parsing Strings for Parameter references. A Token may be a reference to a Parameter,
+ * or it may be an "escaped reference" / non-reference.
+ */
+public interface ParameterToken {
+    /**
+     * @return the 0-based index in the String at which the token begins
+     */
+    int getStartOffset();
+
+    /**
+     * @return the 0-based index in the String at which the token ends
+     */
+    int getEndOffset();
+
+    /**
+     * @return the portion of the input text that corresponds to this token
+     */
+    String getText();
+
+    /**
+     * @return <code>true</code> if this token represents an escape sequence such as ##{param} or ## in the case of ###{param} or ####{param}, false if this
+     * token does not represent an escape sequence.
+     */
+    boolean isEscapeSequence();
+
+    /**
+     * @return <code>true</code> if this token represents a reference to a Parameter. If this method returns <code>true</code>, then this token can be cast
+     * as a {@link ParameterReference}.
+     */
+    boolean isParameterReference();
+
+    /**
+     * Returns the 'value' of the token. If this token is a parameter reference, it will return the value of the
+     * Parameter, according to the given Parameter Context. If this token is an Escape Sequence, it will return the
+     * un-escaped version of the escape sequence.
+     * @param lookup the Parameter Lookup to use for looking up values
+     * @return the value of the Token
+     */
+    String getValue(ParameterLookup lookup);
+}
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ParameterTokenList.java
similarity index 62%
copy from nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java
copy to nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ParameterTokenList.java
index 74dafce..b8f7052 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java
+++ b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ParameterTokenList.java
@@ -14,22 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.nifi.parameter;
 
-package org.apache.nifi.attribute.expression.language;
+import java.util.List;
 
-import java.util.Map;
+public interface ParameterTokenList extends Iterable<ParameterToken> {
 
-import org.apache.nifi.expression.AttributeValueDecorator;
+    String substitute(ParameterLookup parameterLookup);
 
-public class StringLiteralExpression implements Expression {
-    private final String value;
+    String escape();
 
-    public StringLiteralExpression(final String value) {
-        this.value = value;
-    }
+    List<ParameterToken> toList();
 
-    @Override
-    public String evaluate(Map<String, String> variables, AttributeValueDecorator decorator, Map<String, String> stateVariables) {
-        return value;
-    }
+    List<ParameterReference> toReferenceList();
 }
diff --git a/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/StandardParameterReference.java b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/StandardParameterReference.java
new file mode 100644
index 0000000..4f270f1
--- /dev/null
+++ b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/StandardParameterReference.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.parameter;
+
+import java.util.Optional;
+
+public class StandardParameterReference implements ParameterReference {
+    private final String parameterName;
+    private final int startOffset;
+    private final int endOffset;
+    private final String referenceText;
+
+    public StandardParameterReference(final String parameterName, final int startOffset, final int endOffset, final String referenceText) {
+        this.parameterName = parameterName;
+        this.startOffset = startOffset;
+        this.endOffset = endOffset;
+        this.referenceText = referenceText;
+    }
+
+    @Override
+    public String getParameterName() {
+        return parameterName;
+    }
+
+    @Override
+    public int getStartOffset() {
+        return startOffset;
+    }
+
+    @Override
+    public int getEndOffset() {
+        return endOffset;
+    }
+
+    @Override
+    public String getText() {
+        return referenceText;
+    }
+
+    @Override
+    public boolean isEscapeSequence() {
+        return false;
+    }
+
+    @Override
+    public boolean isParameterReference() {
+        return true;
+    }
+
+    @Override
+    public String getValue(final ParameterLookup parameterLookup) {
+        if (parameterLookup == null) {
+            return referenceText;
+        }
+
+        final Optional<Parameter> parameter = parameterLookup.getParameter(parameterName);
+        return parameter.map(Parameter::getValue).orElse(referenceText);
+    }
+}
diff --git a/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/StandardParameterTokenList.java b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/StandardParameterTokenList.java
new file mode 100644
index 0000000..f9c0a0f
--- /dev/null
+++ b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/StandardParameterTokenList.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.parameter;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class StandardParameterTokenList implements ParameterTokenList {
+    private final String input;
+    private final List<ParameterToken> tokens;
+    private final List<ParameterReference> referenceList;
+
+    public StandardParameterTokenList(final String input, final List<ParameterToken> tokens) {
+        this.input = input;
+        this.tokens = tokens;
+        this.referenceList = tokens.stream()
+            .filter(ParameterToken::isParameterReference)
+            .map(token -> (ParameterReference) token)
+            .collect(Collectors.toList());
+    }
+
+    @Override
+    public List<ParameterReference> toReferenceList() {
+        return this.referenceList;
+    }
+
+    @Override
+    public String substitute(final ParameterLookup parameterLookup) {
+        if (input == null) {
+            return null;
+        }
+
+        if (tokens.isEmpty()) {
+            return input;
+        }
+
+        return substitute(reference -> reference.getValue(parameterLookup));
+    }
+
+    @Override
+    public String escape() {
+        if (input == null) {
+            return null;
+        }
+
+        return substitute(reference -> {
+            if (reference.isEscapeSequence()) {
+                if (reference.getText().equals("##")) {
+                    return "####";
+                } else {
+                    return "##" + reference.getText();
+                }
+            } else {
+                return "#" + reference.getText();
+            }
+        });
+    }
+
+    private String substitute(final Function<ParameterToken, String> transform) {
+        final StringBuilder sb = new StringBuilder();
+
+        int lastEndOffset = -1;
+        for (final ParameterToken token : tokens) {
+            final int startOffset = token.getStartOffset();
+
+            sb.append(input, lastEndOffset + 1, startOffset);
+            sb.append(transform.apply(token));
+
+            lastEndOffset = token.getEndOffset();
+        }
+
+        if (input.length() > lastEndOffset + 1) {
+            sb.append(input, lastEndOffset + 1, input.length());
+        }
+
+        return sb.toString();
+    }
+
+    @Override
+    public List<ParameterToken> toList() {
+        return Collections.unmodifiableList(tokens);
+    }
+
+    @Override
+    public Iterator<ParameterToken> iterator() {
+        return tokens.iterator();
+    }
+}
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateEvaluator.java b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/StartCharacterEscape.java
similarity index 57%
copy from nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateEvaluator.java
copy to nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/StartCharacterEscape.java
index 352952e..9ff5096 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateEvaluator.java
+++ b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/StartCharacterEscape.java
@@ -14,32 +14,42 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.attribute.expression.language.evaluation;
+package org.apache.nifi.parameter;
 
-import java.util.Date;
+public class StartCharacterEscape implements ParameterToken {
+    private final int startOffset;
 
-import org.apache.nifi.expression.AttributeExpression.ResultType;
+    public StartCharacterEscape(final int startOffset) {
+        this.startOffset = startOffset;
+    }
+
+    @Override
+    public int getStartOffset() {
+        return startOffset;
+    }
 
-public abstract class DateEvaluator implements Evaluator<Date> {
-    private String token;
+    @Override
+    public int getEndOffset() {
+        return startOffset + 1;
+    }
 
     @Override
-    public ResultType getResultType() {
-        return ResultType.DATE;
+    public String getText() {
+        return "##";
     }
 
     @Override
-    public int getEvaluationsRemaining(final EvaluatorState context) {
-        return 0;
+    public boolean isEscapeSequence() {
+        return true;
     }
 
     @Override
-    public String getToken() {
-        return token;
+    public boolean isParameterReference() {
+        return false;
     }
 
     @Override
-    public void setToken(final String token) {
-        this.token = token;
+    public String getValue(final ParameterLookup parameterLookup) {
+        return "#";
     }
 }
diff --git a/nifi-commons/nifi-parameter/src/test/java/org/apache/nifi/parameter/TestExpressionLanguageAgnosticParameterParser.java b/nifi-commons/nifi-parameter/src/test/java/org/apache/nifi/parameter/TestExpressionLanguageAgnosticParameterParser.java
new file mode 100644
index 0000000..f129c5a
--- /dev/null
+++ b/nifi-commons/nifi-parameter/src/test/java/org/apache/nifi/parameter/TestExpressionLanguageAgnosticParameterParser.java
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.parameter;
+
+import org.junit.Test;
+
+import java.util.List;
+
+import static junit.framework.TestCase.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+public class TestExpressionLanguageAgnosticParameterParser {
+
+    @Test
+    public void testProperReferences() {
+        final ParameterParser parameterParser = new ExpressionLanguageAgnosticParameterParser();
+        final ParameterTokenList references = parameterParser.parseTokens("#{foo}");
+
+        for (final ParameterReference reference : references.toReferenceList()) {
+            assertEquals("foo", reference.getParameterName());
+            assertEquals(0, reference.getStartOffset());
+            assertEquals(5, reference.getEndOffset());
+            assertEquals("#{foo}", reference.getText());
+        }
+
+        List<ParameterReference> referenceList = parameterParser.parseTokens("/#{foo}").toReferenceList();
+        assertEquals(1, referenceList.size());
+
+        ParameterReference reference = referenceList.get(0);
+        assertEquals("foo", reference.getParameterName());
+        assertEquals(1, reference.getStartOffset());
+        assertEquals(6, reference.getEndOffset());
+        assertEquals("#{foo}", reference.getText());
+
+        referenceList = parameterParser.parseTokens("/#{foo}/").toReferenceList();
+        assertEquals(1, referenceList.size());
+        reference = referenceList.get(0);
+        assertEquals("foo", reference.getParameterName());
+        assertEquals(1, reference.getStartOffset());
+        assertEquals(6, reference.getEndOffset());
+        assertEquals("#{foo}", reference.getText());
+
+        referenceList = parameterParser.parseTokens("/#{foo}/#{bar}#{baz}").toReferenceList();
+        assertEquals(3, referenceList.size());
+
+        reference = referenceList.get(0);
+        assertEquals("foo", reference.getParameterName());
+        assertEquals(1, reference.getStartOffset());
+        assertEquals(6, reference.getEndOffset());
+        assertEquals("#{foo}", reference.getText());
+
+        reference = referenceList.get(1);
+        assertEquals("bar", reference.getParameterName());
+        assertEquals(8, reference.getStartOffset());
+        assertEquals(13, reference.getEndOffset());
+        assertEquals("#{bar}", reference.getText());
+
+        reference = referenceList.get(2);
+        assertEquals("baz", reference.getParameterName());
+        assertEquals(14, reference.getStartOffset());
+        assertEquals(19, reference.getEndOffset());
+        assertEquals("#{baz}", reference.getText());
+    }
+
+    @Test
+    public void testEscapeSequences() {
+        final ParameterParser parameterParser = new ExpressionLanguageAgnosticParameterParser();
+        List<ParameterToken> tokenList = parameterParser.parseTokens("#{foo}").toList();
+        assertEquals(1, tokenList.size());
+
+        ParameterToken token = tokenList.get(0);
+        assertTrue(token.isParameterReference());
+        assertEquals("foo", ((ParameterReference) token).getParameterName());
+        assertEquals(0, token.getStartOffset());
+        assertEquals(5, token.getEndOffset());
+        assertEquals("#{foo}", token.getText());
+        assertFalse(token.isEscapeSequence());
+
+        tokenList = parameterParser.parseTokens("##{foo}").toList();
+        assertEquals(1, tokenList.size());
+
+        token = tokenList.get(0);
+        assertFalse(token.isParameterReference());
+        assertEquals(0, token.getStartOffset());
+        assertEquals(6, token.getEndOffset());
+        assertEquals("##{foo}", token.getText());
+        assertTrue(token.isEscapeSequence());
+
+        tokenList = parameterParser.parseTokens("###{foo}").toList();
+        assertEquals(2, tokenList.size());
+
+        token = tokenList.get(0);
+        assertFalse(token.isParameterReference());
+        assertEquals(0, token.getStartOffset());
+        assertEquals(1, token.getEndOffset());
+        assertEquals("##", token.getText());
+        assertTrue(token.isEscapeSequence());
+
+        token = tokenList.get(1);
+        assertTrue(token.isParameterReference());
+        assertEquals("foo", ((ParameterReference) token).getParameterName());
+        assertEquals(2, token.getStartOffset());
+        assertEquals(7, token.getEndOffset());
+        assertFalse(token.isEscapeSequence());
+
+        // Test an escaped # followed by an escaped #{foo}
+        tokenList = parameterParser.parseTokens("####{foo}").toList();
+        assertEquals(2, tokenList.size());
+
+        token = tokenList.get(0);
+        assertFalse(token.isParameterReference());
+        assertEquals(0, token.getStartOffset());
+        assertEquals(1, token.getEndOffset());
+        assertEquals("##", token.getText());
+        assertTrue(token.isEscapeSequence());
+
+        token = tokenList.get(1);
+        assertFalse(token.isParameterReference());
+        assertEquals(2, token.getStartOffset());
+        assertEquals(8, token.getEndOffset());
+        assertTrue(token.isEscapeSequence());
+
+        // Test multiple escaped # followed by a reference of #{foo}
+        tokenList = parameterParser.parseTokens("#####{foo}").toList();
+        assertEquals(3, tokenList.size());
+
+        token = tokenList.get(0);
+        assertFalse(token.isParameterReference());
+        assertEquals(0, token.getStartOffset());
+        assertEquals(1, token.getEndOffset());
+        assertEquals("##", token.getText());
+        assertTrue(token.isEscapeSequence());
+
+        token = tokenList.get(1);
+        assertFalse(token.isParameterReference());
+        assertEquals(2, token.getStartOffset());
+        assertEquals(3, token.getEndOffset());
+        assertEquals("##", token.getText());
+        assertTrue(token.isEscapeSequence());
+
+        token = tokenList.get(2);
+        assertTrue(token.isParameterReference());
+        assertEquals("foo", ((ParameterReference) token).getParameterName());
+        assertEquals(4, token.getStartOffset());
+        assertEquals(9, token.getEndOffset());
+        assertFalse(token.isEscapeSequence());
+    }
+
+    @Test
+    public void testNonReferences() {
+        final ParameterParser parameterParser = new ExpressionLanguageAgnosticParameterParser();
+
+        for (final String input : new String[] {"#foo", "Some text #{blah foo", "#foo}", "#}foo{", "#f{oo}", "#", "##", "###", "####", "#####", "#{", "##{", "###{"}) {
+            assertEquals(0, parameterParser.parseTokens(input).toList().size());
+        }
+    }
+
+    @Test
+    public void testReferenceWithinExpressionLanguage() {
+        final ParameterParser parameterParser = new ExpressionLanguageAgnosticParameterParser();
+        final List<ParameterToken> tokens = parameterParser.parseTokens("${#{hello}:toUpper()}").toList();
+        assertEquals(1, tokens.size());
+    }
+
+    @Test
+    public void testReferenceInsideAndOutsideExpressionLanguage() {
+        final ParameterParser parameterParser = new ExpressionLanguageAgnosticParameterParser();
+        final List<ParameterToken> tokens = parameterParser.parseTokens("#{hello}${#{hello}:toUpper()}#{hello}").toList();
+        assertEquals(3, tokens.size());
+
+        for (final ParameterToken token : tokens) {
+            assertEquals("hello", ((ParameterReference) token).getParameterName());
+        }
+    }
+}
diff --git a/nifi-commons/nifi-parameter/src/test/java/org/apache/nifi/parameter/TestExpressionLanguageAwareParameterParser.java b/nifi-commons/nifi-parameter/src/test/java/org/apache/nifi/parameter/TestExpressionLanguageAwareParameterParser.java
new file mode 100644
index 0000000..d4cf058
--- /dev/null
+++ b/nifi-commons/nifi-parameter/src/test/java/org/apache/nifi/parameter/TestExpressionLanguageAwareParameterParser.java
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.parameter;
+
+import org.junit.Test;
+
+import java.util.List;
+
+import static junit.framework.TestCase.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+public class TestExpressionLanguageAwareParameterParser {
+
+    @Test
+    public void testProperReferences() {
+        final ParameterParser parameterParser = new ExpressionLanguageAwareParameterParser();
+        final ParameterTokenList references = parameterParser.parseTokens("#{foo}");
+
+        for (final ParameterReference reference : references.toReferenceList()) {
+            assertEquals("foo", reference.getParameterName());
+            assertEquals(0, reference.getStartOffset());
+            assertEquals(5, reference.getEndOffset());
+            assertEquals("#{foo}", reference.getText());
+        }
+
+        List<ParameterReference> referenceList = parameterParser.parseTokens("/#{foo}").toReferenceList();
+        assertEquals(1, referenceList.size());
+
+        ParameterReference reference = referenceList.get(0);
+        assertEquals("foo", reference.getParameterName());
+        assertEquals(1, reference.getStartOffset());
+        assertEquals(6, reference.getEndOffset());
+        assertEquals("#{foo}", reference.getText());
+
+        referenceList = parameterParser.parseTokens("/#{foo}/").toReferenceList();
+        assertEquals(1, referenceList.size());
+        reference = referenceList.get(0);
+        assertEquals("foo", reference.getParameterName());
+        assertEquals(1, reference.getStartOffset());
+        assertEquals(6, reference.getEndOffset());
+        assertEquals("#{foo}", reference.getText());
+
+        referenceList = parameterParser.parseTokens("/#{foo}/#{bar}#{baz}").toReferenceList();
+        assertEquals(3, referenceList.size());
+
+        reference = referenceList.get(0);
+        assertEquals("foo", reference.getParameterName());
+        assertEquals(1, reference.getStartOffset());
+        assertEquals(6, reference.getEndOffset());
+        assertEquals("#{foo}", reference.getText());
+
+        reference = referenceList.get(1);
+        assertEquals("bar", reference.getParameterName());
+        assertEquals(8, reference.getStartOffset());
+        assertEquals(13, reference.getEndOffset());
+        assertEquals("#{bar}", reference.getText());
+
+        reference = referenceList.get(2);
+        assertEquals("baz", reference.getParameterName());
+        assertEquals(14, reference.getStartOffset());
+        assertEquals(19, reference.getEndOffset());
+        assertEquals("#{baz}", reference.getText());
+    }
+
+    @Test
+    public void testEscapeSequences() {
+        final ParameterParser parameterParser = new ExpressionLanguageAwareParameterParser();
+        List<ParameterToken> tokenList = parameterParser.parseTokens("#{foo}").toList();
+        assertEquals(1, tokenList.size());
+
+        ParameterToken token = tokenList.get(0);
+        assertTrue(token.isParameterReference());
+        assertEquals("foo", ((ParameterReference) token).getParameterName());
+        assertEquals(0, token.getStartOffset());
+        assertEquals(5, token.getEndOffset());
+        assertEquals("#{foo}", token.getText());
+        assertFalse(token.isEscapeSequence());
+
+        tokenList = parameterParser.parseTokens("##{foo}").toList();
+        assertEquals(1, tokenList.size());
+
+        token = tokenList.get(0);
+        assertFalse(token.isParameterReference());
+        assertEquals(0, token.getStartOffset());
+        assertEquals(6, token.getEndOffset());
+        assertEquals("##{foo}", token.getText());
+        assertTrue(token.isEscapeSequence());
+
+        tokenList = parameterParser.parseTokens("###{foo}").toList();
+        assertEquals(2, tokenList.size());
+
+        token = tokenList.get(0);
+        assertFalse(token.isParameterReference());
+        assertEquals(0, token.getStartOffset());
+        assertEquals(1, token.getEndOffset());
+        assertEquals("##", token.getText());
+        assertTrue(token.isEscapeSequence());
+
+        token = tokenList.get(1);
+        assertTrue(token.isParameterReference());
+        assertEquals("foo", ((ParameterReference) token).getParameterName());
+        assertEquals(2, token.getStartOffset());
+        assertEquals(7, token.getEndOffset());
+        assertFalse(token.isEscapeSequence());
+
+        // Test an escaped # followed by an escaped #{foo}
+        tokenList = parameterParser.parseTokens("####{foo}").toList();
+        assertEquals(2, tokenList.size());
+
+        token = tokenList.get(0);
+        assertFalse(token.isParameterReference());
+        assertEquals(0, token.getStartOffset());
+        assertEquals(1, token.getEndOffset());
+        assertEquals("##", token.getText());
+        assertTrue(token.isEscapeSequence());
+
+        token = tokenList.get(1);
+        assertFalse(token.isParameterReference());
+        assertEquals(2, token.getStartOffset());
+        assertEquals(8, token.getEndOffset());
+        assertTrue(token.isEscapeSequence());
+
+        // Test multiple escaped # followed by a reference of #{foo}
+        tokenList = parameterParser.parseTokens("#####{foo}").toList();
+        assertEquals(3, tokenList.size());
+
+        token = tokenList.get(0);
+        assertFalse(token.isParameterReference());
+        assertEquals(0, token.getStartOffset());
+        assertEquals(1, token.getEndOffset());
+        assertEquals("##", token.getText());
+        assertTrue(token.isEscapeSequence());
+
+        token = tokenList.get(1);
+        assertFalse(token.isParameterReference());
+        assertEquals(2, token.getStartOffset());
+        assertEquals(3, token.getEndOffset());
+        assertEquals("##", token.getText());
+        assertTrue(token.isEscapeSequence());
+
+        token = tokenList.get(2);
+        assertTrue(token.isParameterReference());
+        assertEquals("foo", ((ParameterReference) token).getParameterName());
+        assertEquals(4, token.getStartOffset());
+        assertEquals(9, token.getEndOffset());
+        assertFalse(token.isEscapeSequence());
+    }
+
+    @Test
+    public void testNonReferences() {
+        final ParameterParser parameterParser = new ExpressionLanguageAwareParameterParser();
+
+        for (final String input : new String[] {"#foo", "Some text #{blah foo", "#foo}", "#}foo{", "#f{oo}", "#", "##", "###", "####", "#####", "#{", "##{", "###{"}) {
+            assertEquals(0, parameterParser.parseTokens(input).toList().size());
+        }
+    }
+
+    @Test
+    public void testReferenceWithinExpressionLanguage() {
+        final ParameterParser parameterParser = new ExpressionLanguageAwareParameterParser();
+        final List<ParameterToken> tokens = parameterParser.parseTokens("${#{hello}:toUpper()}").toList();
+        assertEquals(0, tokens.size());
+    }
+
+    @Test
+    public void testReferenceInsideAndOutsideExpressionLanguage() {
+        final ParameterParser parameterParser = new ExpressionLanguageAwareParameterParser();
+        final List<ParameterToken> tokens = parameterParser.parseTokens("#{hello}${#{hello}:toUpper()}#{hello}").toList();
+        assertEquals(2, tokens.size());
+
+        for (final ParameterToken token : tokens) {
+            assertEquals("hello", ((ParameterReference) token).getParameterName());
+        }
+    }
+
+    @Test
+    public void testReferenceOutsideExpressionLanguage() {
+        final ParameterParser parameterParser = new ExpressionLanguageAwareParameterParser();
+        final List<ParameterToken> tokens = parameterParser.parseTokens("#{hello}").toList();
+        assertEquals(1, tokens.size());
+
+        for (final ParameterToken token : tokens) {
+            assertEquals("hello", ((ParameterReference) token).getParameterName());
+        }
+    }
+}
diff --git a/nifi-commons/nifi-parameter/src/test/java/org/apache/nifi/parameter/TestStandardParameterTokenList.java b/nifi-commons/nifi-parameter/src/test/java/org/apache/nifi/parameter/TestStandardParameterTokenList.java
new file mode 100644
index 0000000..bfac611
--- /dev/null
+++ b/nifi-commons/nifi-parameter/src/test/java/org/apache/nifi/parameter/TestStandardParameterTokenList.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.parameter;
+
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestStandardParameterTokenList {
+
+    @Test
+    public void testSubstitute() {
+        final List<ParameterToken> referenceList = new ArrayList<>();
+        referenceList.add(new StandardParameterReference("foo", 0, 5, "#{foo}"));
+
+        final ParameterLookup paramLookup = Mockito.mock(ParameterLookup.class);
+        Mockito.when(paramLookup.getParameter("foo")).thenReturn(Optional.of(new Parameter(new ParameterDescriptor.Builder().name("foo").build(), "bar")));
+        Mockito.when(paramLookup.getParameter("bazz")).thenReturn(Optional.of(new Parameter(new ParameterDescriptor.Builder().name("bazz").build(), "baz")));
+
+        StandardParameterTokenList references = new StandardParameterTokenList("#{foo}", referenceList);
+        assertEquals("bar", references.substitute(paramLookup));
+
+        referenceList.add(new StandardParameterReference("bazz", 6, 12, "#{bazz}"));
+
+        references = new StandardParameterTokenList("#{foo}#{bazz}", referenceList);
+        assertEquals("barbaz", references.substitute(paramLookup));
+
+        references = new StandardParameterTokenList("#{foo}#{bazz}Hello, World!", referenceList);
+        assertEquals("barbazHello, World!", references.substitute(paramLookup));
+
+        referenceList.clear();
+        referenceList.add(new StandardParameterReference("foo", 0, 5, "#{foo}"));
+    }
+
+    @Test
+    public void testSubstituteWithReferenceToNonExistentParameter() {
+        final List<ParameterToken> referenceList = new ArrayList<>();
+        referenceList.add(new StandardParameterReference("foo", 0, 5, "#{foo}"));
+
+        final ParameterLookup paramContext = Mockito.mock(ParameterLookup.class);
+        Mockito.when(paramContext.getParameter(Mockito.anyString())).thenReturn(Optional.empty());
+        final StandardParameterTokenList references = new StandardParameterTokenList("#{foo}", referenceList);
+
+        assertEquals("#{foo}", references.substitute(paramContext));
+    }
+
+    @Test
+    public void testSubstituteWithEscapes() {
+        final List<ParameterToken> referenceList = new ArrayList<>();
+        referenceList.add(new StartCharacterEscape(0));
+        referenceList.add(new EscapedParameterReference(2, 8, "##{foo}"));
+
+        final ParameterLookup paramContext = Mockito.mock(ParameterLookup.class);
+        Mockito.when(paramContext.getParameter("foo")).thenReturn(Optional.of(new Parameter(new ParameterDescriptor.Builder().name("foo").build(), "bar")));
+
+        StandardParameterTokenList references = new StandardParameterTokenList("####{foo}", referenceList);
+        assertEquals("##{foo}", references.substitute(paramContext));
+
+        referenceList.add(new StandardParameterReference("foo", 12, 17, "#{foo}"));
+        references = new StandardParameterTokenList("####{foo}***#{foo}", referenceList);
+        assertEquals("##{foo}***bar", references.substitute(paramContext));
+    }
+
+    @Test
+    public void testEscape() {
+        final List<ParameterToken> referenceList = new ArrayList<>();
+
+        assertEquals("Hello", new StandardParameterTokenList("Hello", referenceList).escape());
+
+        referenceList.add(new StandardParameterReference("abc", 0, 5, "#{abc}"));
+        assertEquals("##{abc}", new StandardParameterTokenList("#{abc}", referenceList).escape());
+
+        referenceList.clear();
+        referenceList.add(new EscapedParameterReference(0, 6, "##{abc}"));
+        assertEquals("####{abc}", new StandardParameterTokenList("##{abc}", referenceList).escape());
+
+        referenceList.clear();
+        referenceList.add(new StartCharacterEscape(0));
+        referenceList.add(new StandardParameterReference("abc", 2, 7, "#{abc}"));
+        assertEquals("######{abc}", new StandardParameterTokenList("###{abc}", referenceList).escape());
+
+        referenceList.clear();
+        referenceList.add(new StartCharacterEscape(0));
+        referenceList.add(new EscapedParameterReference(2, 8, "##{abc}"));
+        assertEquals("########{abc}", new StandardParameterTokenList("####{abc}", referenceList).escape());
+    }
+}
diff --git a/nifi-commons/pom.xml b/nifi-commons/pom.xml
index 9441d8d..fec6996 100644
--- a/nifi-commons/pom.xml
+++ b/nifi-commons/pom.xml
@@ -40,5 +40,6 @@
     	<module>nifi-record</module>
         <module>nifi-record-path</module>
         <module>nifi-metrics</module>
+        <module>nifi-parameter</module>
     </modules>
-</project>
\ No newline at end of file
+</project>
diff --git a/nifi-mock/pom.xml b/nifi-mock/pom.xml
index d725b3e..430aabe 100644
--- a/nifi-mock/pom.xml
+++ b/nifi-mock/pom.xml
@@ -34,6 +34,11 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-parameter</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
             <version>1.10.0-SNAPSHOT</version>
         </dependency>
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockPropertyValue.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockPropertyValue.java
index ec2d08b..209559b 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/MockPropertyValue.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockPropertyValue.java
@@ -16,12 +16,8 @@
  */
 package org.apache.nifi.util;
 
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
 import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.attribute.expression.language.Query;
 import org.apache.nifi.attribute.expression.language.Query.Range;
 import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
@@ -36,6 +32,11 @@ import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.registry.VariableRegistry;
 
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
 public class MockPropertyValue implements PropertyValue {
     private final String rawValue;
     private final Boolean expectExpressions;
@@ -65,7 +66,7 @@ public class MockPropertyValue implements PropertyValue {
 
     private MockPropertyValue(final String rawValue, final ControllerServiceLookup serviceLookup, final PropertyDescriptor propertyDescriptor, final boolean alreadyEvaluated,
             final VariableRegistry variableRegistry) {
-        this.stdPropValue = new StandardPropertyValue(rawValue, serviceLookup, variableRegistry);
+        this.stdPropValue = new StandardPropertyValue(rawValue, serviceLookup, ParameterLookup.EMPTY, variableRegistry);
         this.rawValue = rawValue;
         this.serviceLookup = (MockControllerServiceLookup) serviceLookup;
         this.expectExpressions = propertyDescriptor == null ? null : propertyDescriptor.isExpressionLanguageSupported();
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java
index 72820ee..87389be 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java
@@ -16,12 +16,8 @@
  */
 package org.apache.nifi.util;
 
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
+import org.apache.nifi.parameter.ExpressionLanguageAgnosticParameterParser;
+import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.attribute.expression.language.Query;
 import org.apache.nifi.attribute.expression.language.Query.Range;
 import org.apache.nifi.attribute.expression.language.StandardExpressionLanguageCompiler;
@@ -32,8 +28,18 @@ import org.apache.nifi.components.state.StateManager;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.ControllerServiceLookup;
 import org.apache.nifi.expression.ExpressionLanguageCompiler;
+import org.apache.nifi.parameter.ParameterParser;
+import org.apache.nifi.parameter.ParameterReference;
+import org.apache.nifi.parameter.ExpressionLanguageAwareParameterParser;
 import org.apache.nifi.registry.VariableRegistry;
 
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
 
 public class MockValidationContext extends MockControllerServiceLookup implements ValidationContext, ControllerServiceLookup {
 
@@ -70,7 +76,7 @@ public class MockValidationContext extends MockControllerServiceLookup implement
 
     @Override
     public ExpressionLanguageCompiler newExpressionLanguageCompiler() {
-        return new StandardExpressionLanguageCompiler(variableRegistry);
+        return new StandardExpressionLanguageCompiler(variableRegistry, ParameterLookup.EMPTY);
     }
 
     @Override
@@ -160,4 +166,23 @@ public class MockValidationContext extends MockControllerServiceLookup implement
         return "unit test";
     }
 
+    @Override
+    public Collection<String> getReferencedParameters(final String propertyName) {
+        final String rawPropertyValue = context.getProperty(propertyName).getValue();
+        final boolean elSupported = isExpressionLanguageSupported(propertyName);
+
+        final ParameterParser parser = elSupported ? new ExpressionLanguageAwareParameterParser() : new ExpressionLanguageAgnosticParameterParser();
+
+        final List<ParameterReference> references = parser.parseTokens(rawPropertyValue).toReferenceList();
+        return references.stream()
+            .map(ParameterReference::getParameterName)
+            .collect(Collectors.toList());
+    }
+
+    @Override
+    public boolean isParameterDefined(final String parameterName) {
+        // TODO: Implement
+        return false;
+    }
+
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/AffectedComponentDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/AffectedComponentDTO.java
index dd7c6c4..ebfcadd 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/AffectedComponentDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/AffectedComponentDTO.java
@@ -116,4 +116,9 @@ public class AffectedComponentDTO {
     public void setState(String state) {
         this.state = state;
     }
+
+    @Override
+    public String toString() {
+        return "AffectedComponent[type=" + referenceType + ", id=" + id + ", name=" + name + ", processGroup=" + processGroupId + "]";
+    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/AsynchronousRequestDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/AsynchronousRequestDTO.java
new file mode 100644
index 0000000..180d4f5
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/AsynchronousRequestDTO.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.api.dto;
+
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.Date;
+import java.util.List;
+
+public abstract class AsynchronousRequestDTO<T extends UpdateStepDTO> {
+    private String requestId;
+    private String uri;
+    private Date submissionTime;
+    private Date lastUpdated;
+    private boolean complete = false;
+    private String failureReason;
+    private int percentCompleted;
+    private String state;
+    private List<T> updateSteps;
+
+
+    @ApiModelProperty(value = "The ID of the request", readOnly = true)
+    public String getRequestId() {
+        return requestId;
+    }
+
+    public void setRequestId(final String requestId) {
+        this.requestId = requestId;
+    }
+
+    @ApiModelProperty(value = "The URI for the request", readOnly = true)
+    public String getUri() {
+        return uri;
+    }
+
+    public void setUri(final String uri) {
+        this.uri = uri;
+    }
+
+    @ApiModelProperty(value = "The timestamp of when the request was submitted", readOnly =  true)
+    public Date getSubmissionTime() {
+        return submissionTime;
+    }
+
+    public void setSubmissionTime(final Date submissionTime) {
+        this.submissionTime = submissionTime;
+    }
+
+    @ApiModelProperty(value = "The timestamp of when the request was last updated", readOnly = true)
+    public Date getLastUpdated() {
+        return lastUpdated;
+    }
+
+    public void setLastUpdated(final Date lastUpdated) {
+        this.lastUpdated = lastUpdated;
+    }
+
+    @ApiModelProperty(value = "Whether or not the request is completed", readOnly = true)
+    public boolean isComplete() {
+        return complete;
+    }
+
+    public void setComplete(final boolean complete) {
+        this.complete = complete;
+    }
+
+    @ApiModelProperty(value = "The reason for the request failing, or null if the request has not failed", readOnly = true)
+    public String getFailureReason() {
+        return failureReason;
+    }
+
+    public void setFailureReason(final String failureReason) {
+        this.failureReason = failureReason;
+    }
+
+    @ApiModelProperty(value = "A value between 0 and 100 (inclusive) indicating how close the request is to completion", readOnly = true)
+    public int getPercentCompleted() {
+        return percentCompleted;
+    }
+
+    public void setPercentCompleted(final int percentCompleted) {
+        this.percentCompleted = percentCompleted;
+    }
+
+    @ApiModelProperty(value = "A description of the current state of the request", readOnly = true)
+    public String getState() {
+        return state;
+    }
+
+    public void setState(final String state) {
+        this.state = state;
+    }
+
+    @ApiModelProperty(value = "The steps that are required in order to complete the request, along with the status of each", readOnly = true)
+    public List<T> getUpdateSteps() {
+        return updateSteps;
+    }
+
+    public void setUpdateSteps(List<T> updateSteps) {
+        this.updateSteps = updateSteps;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ComponentValidationResultDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ComponentValidationResultDTO.java
new file mode 100644
index 0000000..3da52b5
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ComponentValidationResultDTO.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.api.dto;
+
+import io.swagger.annotations.ApiModelProperty;
+
+import javax.xml.bind.annotation.XmlType;
+import java.util.Collection;
+
+@XmlType(name = "componentValidationResult")
+public class ComponentValidationResultDTO extends AffectedComponentDTO {
+    private Boolean currentlyValid;
+    private Boolean resultsValid;
+    private Collection<String> resultantValidationErrors;
+
+    @ApiModelProperty("Whether or not the component is currently valid")
+    public Boolean getCurrentlyValid() {
+        return currentlyValid;
+    }
+
+    public void setCurrentlyValid(final Boolean currentlyValid) {
+        this.currentlyValid = currentlyValid;
+    }
+
+    @ApiModelProperty("Whether or not the component will be valid if the Parameter Context is changed")
+    public Boolean getResultsValid() {
+        return resultsValid;
+    }
+
+    public void setResultsValid(final Boolean resultsValid) {
+        this.resultsValid = resultsValid;
+    }
+
+    @ApiModelProperty("The validation errors that will apply to the component if the Parameter Context is changed")
+    public Collection<String> getResultantValidationErrors() {
+        return resultantValidationErrors;
+    }
+
+    public void setResultantValidationErrors(final Collection<String> resultantValidationErrors) {
+        this.resultantValidationErrors = resultantValidationErrors;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterContextDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterContextDTO.java
new file mode 100644
index 0000000..d40fa31
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterContextDTO.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.api.dto;
+
+import io.swagger.annotations.ApiModelProperty;
+import org.apache.nifi.web.api.entity.ParameterEntity;
+import org.apache.nifi.web.api.entity.ProcessGroupEntity;
+
+import javax.xml.bind.annotation.XmlType;
+import java.util.Set;
+
+@XmlType(name = "parameterContext")
+public class ParameterContextDTO {
+    private String identifier;
+    private String name;
+    private String description;
+    private Set<ParameterEntity> parameters;
+    private Set<ProcessGroupEntity> boundProcessGroups;
+
+    public void setId(String id) {
+        this.identifier = id;
+    }
+
+    @ApiModelProperty(value = "The ID the Parameter Context.", readOnly = true)
+    public String getId() {
+        return identifier;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    @ApiModelProperty("The Name of the Parameter Context.")
+    public String getName() {
+        return name;
+    }
+
+    public void setDescription(String description) {
+        this.description = description;
+    }
+
+    @ApiModelProperty("The Description of the Parameter Context.")
+    public String getDescription() {
+        return description;
+    }
+
+    public void setParameters(final Set<ParameterEntity> parameters) {
+        this.parameters = parameters;
+    }
+
+    @ApiModelProperty("The Parameters for the Parameter Context")
+    public Set<ParameterEntity> getParameters() {
+        return parameters;
+    }
+
+    public void setBoundProcessGroups(final Set<ProcessGroupEntity> boundProcessGroups) {
+        this.boundProcessGroups = boundProcessGroups;
+    }
+
+    @ApiModelProperty(value = "The Process Groups that are bound to this Parameter Context", readOnly = true)
+    public Set<ProcessGroupEntity> getBoundProcessGroups() {
+        return boundProcessGroups;
+    }
+
+    @Override
+    public String toString() {
+        return "ParameterContext[id=" + identifier + ", name=" + name + ", parameters=" + parameters + "]";
+    }
+}
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterContextReferenceDTO.java
similarity index 63%
copy from nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterContextReferenceDTO.java
index 74dafce..9d6241f 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterContextReferenceDTO.java
@@ -14,22 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.nifi.web.api.dto;
 
-package org.apache.nifi.attribute.expression.language;
+import io.swagger.annotations.ApiModelProperty;
 
-import java.util.Map;
+import javax.xml.bind.annotation.XmlType;
 
-import org.apache.nifi.expression.AttributeValueDecorator;
+@XmlType(name = "parameterContextReference")
+public class ParameterContextReferenceDTO {
+    private String id;
 
-public class StringLiteralExpression implements Expression {
-    private final String value;
-
-    public StringLiteralExpression(final String value) {
-        this.value = value;
+    public void setId(String id) {
+        this.id = id;
     }
 
-    @Override
-    public String evaluate(Map<String, String> variables, AttributeValueDecorator decorator, Map<String, String> stateVariables) {
-        return value;
+    @ApiModelProperty("The ID of the Parameter Context")
+    public String getId() {
+        return id;
     }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterContextUpdateRequestDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterContextUpdateRequestDTO.java
new file mode 100644
index 0000000..76acc8d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterContextUpdateRequestDTO.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.api.dto;
+
+import io.swagger.annotations.ApiModelProperty;
+import org.apache.nifi.web.api.entity.AffectedComponentEntity;
+
+import javax.xml.bind.annotation.XmlType;
+import java.util.Set;
+
+@XmlType(name = "parameterContextUpdateRequest")
+public class ParameterContextUpdateRequestDTO extends AsynchronousRequestDTO<ParameterContextUpdateStepDTO> {
+    private ParameterContextDTO parameterContext;
+    private Set<AffectedComponentEntity> affectedComponents;
+
+    @ApiModelProperty(value = "The Parameter Context that is being operated on. This may not be populated until the request has successfully completed.", readOnly = true)
+    public ParameterContextDTO getParameterContext() {
+        return parameterContext;
+    }
+
+    public void setParameterContext(final ParameterContextDTO parameterContext) {
+        this.parameterContext = parameterContext;
+    }
+
+    @ApiModelProperty(value = "The components that are affected by the update.", readOnly = true)
+    public Set<AffectedComponentEntity> getAffectedComponents() {
+        return affectedComponents;
+    }
+
+    public void setAffectedComponents(final Set<AffectedComponentEntity> affectedComponents) {
+        this.affectedComponents = affectedComponents;
+    }
+}
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterContextUpdateStepDTO.java
similarity index 62%
copy from nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterContextUpdateStepDTO.java
index 74dafce..e105958 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterContextUpdateStepDTO.java
@@ -14,22 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.nifi.web.api.dto;
 
-package org.apache.nifi.attribute.expression.language;
+import javax.xml.bind.annotation.XmlType;
 
-import java.util.Map;
-
-import org.apache.nifi.expression.AttributeValueDecorator;
-
-public class StringLiteralExpression implements Expression {
-    private final String value;
-
-    public StringLiteralExpression(final String value) {
-        this.value = value;
-    }
-
-    @Override
-    public String evaluate(Map<String, String> variables, AttributeValueDecorator decorator, Map<String, String> stateVariables) {
-        return value;
-    }
+@XmlType(name = "parameterContextUpdateStep")
+public class ParameterContextUpdateStepDTO extends UpdateStepDTO {
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterContextValidationRequestDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterContextValidationRequestDTO.java
new file mode 100644
index 0000000..c475286
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterContextValidationRequestDTO.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.api.dto;
+
+import io.swagger.annotations.ApiModelProperty;
+import org.apache.nifi.web.api.entity.ComponentValidationResultsEntity;
+
+import javax.xml.bind.annotation.XmlType;
+
+@XmlType(name = "parameterContextValidationRequest")
+public class ParameterContextValidationRequestDTO extends AsynchronousRequestDTO<ParameterContextValidationStepDTO> {
+    private ParameterContextDTO parameterContext;
+    private ComponentValidationResultsEntity componentValidationResults;
+
+    @ApiModelProperty(value = "The Validation Results that were calculated for each component. This value may not be set until the request completes.", readOnly = true)
+    public ComponentValidationResultsEntity getComponentValidationResults() {
+        return componentValidationResults;
+    }
+
+    public void setComponentValidationResults(final ComponentValidationResultsEntity componentValidationResults) {
+        this.componentValidationResults = componentValidationResults;
+    }
+
+    @ApiModelProperty("The Parameter Context that is being operated on.")
+    public ParameterContextDTO getParameterContext() {
+        return parameterContext;
+    }
+
+    public void setParameterContext(final ParameterContextDTO parameterContext) {
+        this.parameterContext = parameterContext;
+    }
+
+}
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterContextValidationResultsDTO.java
similarity index 58%
copy from nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterContextValidationResultsDTO.java
index 74dafce..b4fb1c1 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterContextValidationResultsDTO.java
@@ -14,22 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.nifi.web.api.dto;
 
-package org.apache.nifi.attribute.expression.language;
+import javax.xml.bind.annotation.XmlType;
+import java.util.List;
 
-import java.util.Map;
+@XmlType(name = "parameterContextValidationResults")
+public class ParameterContextValidationResultsDTO {
+    private List<ComponentValidationResultDTO> componentValidationResults;
 
-import org.apache.nifi.expression.AttributeValueDecorator;
-
-public class StringLiteralExpression implements Expression {
-    private final String value;
-
-    public StringLiteralExpression(final String value) {
-        this.value = value;
+    public void setComponentValidationResults(List<ComponentValidationResultDTO> componentValidationResults) {
+        this.componentValidationResults = componentValidationResults;
     }
 
-    @Override
-    public String evaluate(Map<String, String> variables, AttributeValueDecorator decorator, Map<String, String> stateVariables) {
-        return value;
+    public List<ComponentValidationResultDTO> getComponentValidationResults() {
+        return componentValidationResults;
     }
 }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterContextValidationStepDTO.java
similarity index 61%
copy from nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterContextValidationStepDTO.java
index 74dafce..76af752 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StringLiteralExpression.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterContextValidationStepDTO.java
@@ -14,22 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.nifi.web.api.dto;
 
-package org.apache.nifi.attribute.expression.language;
+import javax.xml.bind.annotation.XmlType;
 
-import java.util.Map;
-
-import org.apache.nifi.expression.AttributeValueDecorator;
-
-public class StringLiteralExpression implements Expression {
-    private final String value;
-
-    public StringLiteralExpression(final String value) {
-        this.value = value;
-    }
-
-    @Override
-    public String evaluate(Map<String, String> variables, AttributeValueDecorator decorator, Map<String, String> stateVariables) {
-        return value;
-    }
+@XmlType(name = "parameterContextValidationStep")
+public class ParameterContextValidationStepDTO extends UpdateStepDTO {
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterDTO.java
new file mode 100644
index 0000000..4cbeb6f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterDTO.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.api.dto;
+
+import io.swagger.annotations.ApiModelProperty;
+import org.apache.nifi.web.api.entity.AffectedComponentEntity;
+
+import javax.xml.bind.annotation.XmlType;
+import java.util.Set;
+
+@XmlType(name = "parameter")
+public class ParameterDTO {
+    private String name;
+    private String description;
+    private Boolean sensitive;
+    private String value;
+    private Set<AffectedComponentEntity> referencingComponents;
+
+    @ApiModelProperty("The name of the Parameter")
+    public String getName() {
+        return name;
+    }
+
+    public void setName(final String name) {
+        this.name = name;
+    }
+
+    @ApiModelProperty("The description of the Parameter")
+    public String getDescription() {
+        return description;
+    }
+
+    public void setDescription(final String description) {
+        this.description = description;
+    }
+
+    @ApiModelProperty("Whether or not the Parameter is sensitive")
+    public Boolean getSensitive() {
+        return sensitive;
+    }
+
+    public void setSensitive(final Boolean sensitive) {
+        this.sensitive = sensitive;
+    }
+
+    @ApiModelProperty("The value of the Parameter")
+    public String getValue() {
+        return value;
+    }
+
+    public void setValue(final String value) {
+        this.value = value;
+    }
+
+    @ApiModelProperty("The set of all components in the flow that are referencing this Parameter")
+    public Set<AffectedComponentEntity> getReferencingComponents() {
+        return referencingComponents;
+    }
+
+    public void setReferencingComponents(final Set<AffectedComponentEntity> referencingComponents) {
+        this.referencingComponents = referencingComponents;
+    }
+
+    @Override
+    public String toString() {
+        return "ParameterDTO[name=" + name + ", sensitive=" + sensitive + ", value=" + (sensitive ? "********" : value) + "]";
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.java
index 3fe61d4..f5a4e35 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.java
@@ -32,6 +32,7 @@ public class ProcessGroupDTO extends ComponentDTO {
     private String comments;
     private Map<String, String> variables;
     private VersionControlInformationDTO versionControlInformation;
+    private ParameterContextReferenceDTO parameterContext;
 
     private Integer runningCount;
     private Integer stoppedCount;
@@ -341,4 +342,13 @@ public class ProcessGroupDTO extends ComponentDTO {
     public void setVersionControlInformation(final VersionControlInformationDTO versionControlInformation) {
         this.versionControlInformation = versionControlInformation;
     }
+
+    @ApiModelProperty("The Parameter Context that this Process Group is bound to.")
+    public ParameterContextReferenceDTO getParameterContext() {
+        return parameterContext;
+    }
+
+    public void setParameterContext(final ParameterContextReferenceDTO parameterContext) {
+        this.parameterContext = parameterContext;
+    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/TemplateDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/TemplateDTO.java
index 02792af..da358a6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/TemplateDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/TemplateDTO.java
@@ -29,7 +29,7 @@ import java.util.Date;
  */
 @XmlRootElement(name = "template")
 public class TemplateDTO {
-    public static final String MAX_ENCODING_VERSION = "1.2";
+    public static final String MAX_ENCODING_VERSION = "1.3";
 
     private String uri;
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateStepDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/UpdateStepDTO.java
similarity index 90%
copy from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateStepDTO.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/UpdateStepDTO.java
index 81e0057..25e3f1f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateStepDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/UpdateStepDTO.java
@@ -14,22 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.nifi.web.api.dto;
 
 import io.swagger.annotations.ApiModelProperty;
 
-import javax.xml.bind.annotation.XmlType;
-
-@XmlType(name = "varaibleRegistryUpdateStep")
-public class VariableRegistryUpdateStepDTO {
+public abstract class UpdateStepDTO {
     private String description;
     private boolean complete;
     private String failureReason;
 
-    public VariableRegistryUpdateStepDTO() {
-    }
-
     @ApiModelProperty(value = "Explanation of what happens in this step", readOnly = true)
     public String getDescription() {
         return description;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateRequestDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateRequestDTO.java
index 24400e6..9e4bb61 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateRequestDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateRequestDTO.java
@@ -18,25 +18,14 @@
 package org.apache.nifi.web.api.dto;
 
 import io.swagger.annotations.ApiModelProperty;
-import org.apache.nifi.web.api.dto.util.TimestampAdapter;
 import org.apache.nifi.web.api.entity.AffectedComponentEntity;
 
 import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import java.util.Date;
-import java.util.List;
 import java.util.Set;
 
 @XmlType(name = "variableRegistryUpdateRequest")
-public class VariableRegistryUpdateRequestDTO {
-    private String requestId;
+public class VariableRegistryUpdateRequestDTO extends AsynchronousRequestDTO<VariableRegistryUpdateStepDTO> {
     private String processGroupId;
-    private String uri;
-    private Date submissionTime;
-    private Date lastUpdated;
-    private boolean complete = false;
-    private String failureReason;
-    private List<VariableRegistryUpdateStepDTO> updateSteps;
     private Set<AffectedComponentEntity> affectedComponents;
 
     @ApiModelProperty("The unique ID of the Process Group that the variable registry belongs to")
@@ -48,71 +37,6 @@ public class VariableRegistryUpdateRequestDTO {
         this.processGroupId = processGroupId;
     }
 
-    @ApiModelProperty(value = "The unique ID of this request.", readOnly = true)
-    public String getRequestId() {
-        return requestId;
-    }
-
-    public void setRequestId(String requestId) {
-        this.requestId = requestId;
-    }
-
-    @ApiModelProperty(value = "The URI for future requests to this drop request.", readOnly = true)
-    public String getUri() {
-        return uri;
-    }
-
-    public void setUri(String uri) {
-        this.uri = uri;
-    }
-
-    @XmlJavaTypeAdapter(TimestampAdapter.class)
-    @ApiModelProperty(value = "The time at which this request was submitted.", dataType = "string", readOnly = true)
-    public Date getSubmissionTime() {
-        return submissionTime;
-    }
-
-    public void setSubmissionTime(Date submissionTime) {
-        this.submissionTime = submissionTime;
-    }
-
-    @XmlJavaTypeAdapter(TimestampAdapter.class)
-    @ApiModelProperty(value = "The last time this request was updated.", dataType = "string", readOnly = true)
-    public Date getLastUpdated() {
-        return lastUpdated;
-    }
-
-    public void setLastUpdated(Date lastUpdated) {
-        this.lastUpdated = lastUpdated;
-    }
-
-    @ApiModelProperty(value = "The steps that are required in order to complete the request, along with the status of each", readOnly = true)
-    public List<VariableRegistryUpdateStepDTO> getUpdateSteps() {
-        return updateSteps;
-    }
-
-    public void setUpdateSteps(List<VariableRegistryUpdateStepDTO> updateSteps) {
-        this.updateSteps = updateSteps;
-    }
-
-    @ApiModelProperty(value = "Whether or not this request has completed", readOnly = true)
-    public boolean isComplete() {
-        return complete;
-    }
-
-    public void setComplete(boolean complete) {
-        this.complete = complete;
-    }
-
-    @ApiModelProperty(value = "An explanation of why this request failed, or null if this request has not failed", readOnly = true)
-    public String getFailureReason() {
-        return failureReason;
-    }
-
-    public void setFailureReason(String reason) {
-        this.failureReason = reason;
-    }
-
     @ApiModelProperty(value = "A set of all components that will be affected if the value of this variable is changed", readOnly = true)
     public Set<AffectedComponentEntity> getAffectedComponents() {
         return affectedComponents;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateStepDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateStepDTO.java
index 81e0057..2dd172c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateStepDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateStepDTO.java
@@ -17,43 +17,9 @@
 
 package org.apache.nifi.web.api.dto;
 
-import io.swagger.annotations.ApiModelProperty;
-
 import javax.xml.bind.annotation.XmlType;
 
 @XmlType(name = "varaibleRegistryUpdateStep")
-public class VariableRegistryUpdateStepDTO {
-    private String description;
-    private boolean complete;
-    private String failureReason;
-
-    public VariableRegistryUpdateStepDTO() {
-    }
-
-    @ApiModelProperty(value = "Explanation of what happens in this step", readOnly = true)
-    public String getDescription() {
-        return description;
-    }
-
-    public void setDescription(String description) {
-        this.description = description;
-    }
-
-    @ApiModelProperty(value = "Whether or not this step has completed", readOnly = true)
-    public boolean isComplete() {
-        return complete;
-    }
-
-    public void setComplete(boolean complete) {
-        this.complete = complete;
-    }
-
-    @ApiModelProperty(value = "An explanation of why this step failed, or null if this step did not fail", readOnly = true)
-    public String getFailureReason() {
-        return failureReason;
-    }
+public class VariableRegistryUpdateStepDTO extends UpdateStepDTO {
 
-    public void setFailureReason(String reason) {
-        this.failureReason = reason;
-    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/flow/ProcessGroupFlowDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/flow/ProcessGroupFlowDTO.java
index 2ef6115..479375e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/flow/ProcessGroupFlowDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/flow/ProcessGroupFlowDTO.java
@@ -33,6 +33,7 @@ public class ProcessGroupFlowDTO {
     private String id;
     private String uri;
     private String parentGroupId;
+    private String parameterContextId;
     private FlowBreadcrumbEntity breadcrumb;
     private FlowDTO flow;
     private Date lastRefreshed;
@@ -125,4 +126,13 @@ public class ProcessGroupFlowDTO {
     public void setLastRefreshed(Date lastRefreshed) {
         this.lastRefreshed = lastRefreshed;
     }
+
+    @ApiModelProperty("The ID of the Parameter Context, or null if no Parameter Context has been bound to the Process Group")
+    public String getParameterContextId() {
+        return parameterContextId;
+    }
+
+    public void setParameterContextId(String parameterContextId) {
+        this.parameterContextId = parameterContextId;
+    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/AffectedComponentEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/AffectedComponentEntity.java
index e0d8496..efb7505 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/AffectedComponentEntity.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/AffectedComponentEntity.java
@@ -41,4 +41,9 @@ public class AffectedComponentEntity extends ComponentEntity implements Permissi
     public void setComponent(AffectedComponentDTO component) {
         this.component = component;
     }
+
+    @Override
+    public String toString() {
+        return component == null ? "AffectedComponent[No Component]" : component.toString();
+    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/AffectedComponentEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ComponentValidationResultEntity.java
similarity index 60%
copy from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/AffectedComponentEntity.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ComponentValidationResultEntity.java
index e0d8496..d7a5718 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/AffectedComponentEntity.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ComponentValidationResultEntity.java
@@ -16,29 +16,23 @@
  */
 package org.apache.nifi.web.api.entity;
 
-import org.apache.nifi.web.api.dto.AffectedComponentDTO;
+import org.apache.nifi.web.api.dto.ComponentValidationResultDTO;
 
 import javax.xml.bind.annotation.XmlRootElement;
 
-/**
- * A serialized representation of this class can be placed in the entity body of a response to the API.
- * This particular entity holds a reference to component that references a variable.
- */
-@XmlRootElement(name = "affectComponentEntity")
-public class AffectedComponentEntity extends ComponentEntity implements Permissible<AffectedComponentDTO> {
-
-    private AffectedComponentDTO component;
+@XmlRootElement(name = "componentValidationResultEntity")
+public class ComponentValidationResultEntity extends ComponentEntity implements Permissible<ComponentValidationResultDTO> {
+    private ComponentValidationResultDTO validationResult;
 
     /**
      * @return variable referencing components that is being serialized
      */
-    @Override
-    public AffectedComponentDTO getComponent() {
-        return component;
+    public ComponentValidationResultDTO getComponent() {
+        return validationResult;
     }
 
-    @Override
-    public void setComponent(AffectedComponentDTO component) {
-        this.component = component;
+    public void setComponent(ComponentValidationResultDTO component) {
+        this.validationResult = component;
     }
+
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/AffectedComponentEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ComponentValidationResultsEntity.java
similarity index 54%
copy from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/AffectedComponentEntity.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ComponentValidationResultsEntity.java
index e0d8496..7648a23 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/AffectedComponentEntity.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ComponentValidationResultsEntity.java
@@ -16,29 +16,21 @@
  */
 package org.apache.nifi.web.api.entity;
 
-import org.apache.nifi.web.api.dto.AffectedComponentDTO;
+import io.swagger.annotations.ApiModelProperty;
 
 import javax.xml.bind.annotation.XmlRootElement;
+import java.util.List;
 
-/**
- * A serialized representation of this class can be placed in the entity body of a response to the API.
- * This particular entity holds a reference to component that references a variable.
- */
-@XmlRootElement(name = "affectComponentEntity")
-public class AffectedComponentEntity extends ComponentEntity implements Permissible<AffectedComponentDTO> {
-
-    private AffectedComponentDTO component;
+@XmlRootElement(name = "componentValidationResults")
+public class ComponentValidationResultsEntity extends Entity {
+    private List<ComponentValidationResultEntity> componentValidationResults;
 
-    /**
-     * @return variable referencing components that is being serialized
-     */
-    @Override
-    public AffectedComponentDTO getComponent() {
-        return component;
+    @ApiModelProperty("A List of ComponentValidationResultEntity, one for each component that is validated")
+    public List<ComponentValidationResultEntity> getValidationResults() {
+        return componentValidationResults;
     }
 
-    @Override
-    public void setComponent(AffectedComponentDTO component) {
-        this.component = component;
+    public void setValidationResults(final List<ComponentValidationResultEntity> componentValidationResults) {
+        this.componentValidationResults = componentValidationResults;
     }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/AffectedComponentEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ParameterContextEntity.java
similarity index 55%
copy from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/AffectedComponentEntity.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ParameterContextEntity.java
index e0d8496..e74ec10 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/AffectedComponentEntity.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ParameterContextEntity.java
@@ -16,29 +16,26 @@
  */
 package org.apache.nifi.web.api.entity;
 
-import org.apache.nifi.web.api.dto.AffectedComponentDTO;
+import io.swagger.annotations.ApiModelProperty;
+import org.apache.nifi.web.api.dto.ParameterContextDTO;
 
 import javax.xml.bind.annotation.XmlRootElement;
 
-/**
- * A serialized representation of this class can be placed in the entity body of a response to the API.
- * This particular entity holds a reference to component that references a variable.
- */
-@XmlRootElement(name = "affectComponentEntity")
-public class AffectedComponentEntity extends ComponentEntity implements Permissible<AffectedComponentDTO> {
+@XmlRootElement(name = "parameterContextEntity")
+public class ParameterContextEntity extends ComponentEntity {
+    private ParameterContextDTO parameterContextDTO;
 
-    private AffectedComponentDTO component;
+    @ApiModelProperty("The Parameter Context")
+    public ParameterContextDTO getComponent() {
+        return parameterContextDTO;
+    }
 
-    /**
-     * @return variable referencing components that is being serialized
-     */
-    @Override
-    public AffectedComponentDTO getComponent() {
-        return component;
+    public void setComponent(ParameterContextDTO parameterContext) {
+        this.parameterContextDTO = parameterContext;
     }
 
     @Override
-    public void setComponent(AffectedComponentDTO component) {
-        this.component = component;
+    public String toString() {
+        return parameterContextDTO == null ? "ParameterContextEntity[No Component]" : parameterContextDTO.toString();
     }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ParameterContextUpdateRequestEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ParameterContextUpdateRequestEntity.java
new file mode 100644
index 0000000..a86cba8
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ParameterContextUpdateRequestEntity.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.api.entity;
+
+import io.swagger.annotations.ApiModelProperty;
+import org.apache.nifi.web.api.dto.ParameterContextUpdateRequestDTO;
+import org.apache.nifi.web.api.dto.RevisionDTO;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+@XmlRootElement(name = "parameterContextUpdateRequestEntity")
+public class ParameterContextUpdateRequestEntity extends Entity {
+    private RevisionDTO parameterContextRevision;
+    private ParameterContextUpdateRequestDTO request;
+
+    @ApiModelProperty("The Revision of the Parameter Context")
+    public RevisionDTO getParameterContextRevision() {
+        return parameterContextRevision;
+    }
+
+    public void setParameterContextRevision(final RevisionDTO parameterContextRevision) {
+        this.parameterContextRevision = parameterContextRevision;
+    }
+
+    @ApiModelProperty("The Update Request")
+    public ParameterContextUpdateRequestDTO getRequest() {
+        return request;
+    }
+
+    public void setRequest(final ParameterContextUpdateRequestDTO request) {
+        this.request = request;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ParameterContextValidationRequestEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ParameterContextValidationRequestEntity.java
new file mode 100644
index 0000000..a5aeece
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ParameterContextValidationRequestEntity.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.api.entity;
+
+import io.swagger.annotations.ApiModelProperty;
+import org.apache.nifi.web.api.dto.ParameterContextValidationRequestDTO;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+@XmlRootElement(name = "parameterContextValidationRequestEntity")
+public class ParameterContextValidationRequestEntity extends Entity {
+    private ParameterContextValidationRequestDTO request;
+    private Boolean disconnectedNodeAcknowledged;
+
+    @ApiModelProperty("The Update Request")
+    public ParameterContextValidationRequestDTO getRequest() {
+        return request;
+    }
+
+    public void setRequest(final ParameterContextValidationRequestDTO request) {
+        this.request = request;
+    }
+
+    @ApiModelProperty("Acknowledges that this node is disconnected to allow for mutable requests to proceed.")
+    public Boolean isDisconnectedNodeAcknowledged() {
+        return disconnectedNodeAcknowledged;
+    }
+
+    public void setDisconnectedNodeAcknowledged(Boolean disconnectedNodeAcknowledged) {
+        this.disconnectedNodeAcknowledged = disconnectedNodeAcknowledged;
+    }
+
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ParameterContextsEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ParameterContextsEntity.java
new file mode 100644
index 0000000..3f581af
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ParameterContextsEntity.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.api.entity;
+
+import io.swagger.annotations.ApiModelProperty;
+import org.apache.nifi.web.api.dto.util.TimeAdapter;
+
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
+import java.util.Date;
+import java.util.Set;
+
+@XmlRootElement(name = "parameterContexts")
+public class ParameterContextsEntity extends Entity {
+    private Set<ParameterContextEntity> parameterContexts;
+    private Date currentTime;
+
+    @ApiModelProperty("The Parameter Contexts")
+    public Set<ParameterContextEntity> getParameterContexts() {
+        return parameterContexts;
+    }
+
+    public void setParameterContexts(final Set<ParameterContextEntity> parameterContexts) {
+        this.parameterContexts = parameterContexts;
+    }
+
+    @XmlJavaTypeAdapter(TimeAdapter.class)
+    @ApiModelProperty(
+        value = "The current time on the system.",
+        dataType = "string",
+        readOnly = true
+    )
+    public Date getCurrentTime() {
+        return currentTime;
+    }
+
+    public void setCurrentTime(Date currentTime) {
+        this.currentTime = currentTime;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/AffectedComponentEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ParameterEntity.java
similarity index 53%
copy from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/AffectedComponentEntity.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ParameterEntity.java
index e0d8496..e77c0ae 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/AffectedComponentEntity.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ParameterEntity.java
@@ -16,29 +16,34 @@
  */
 package org.apache.nifi.web.api.entity;
 
-import org.apache.nifi.web.api.dto.AffectedComponentDTO;
+import io.swagger.annotations.ApiModelProperty;
+import org.apache.nifi.web.api.dto.ParameterDTO;
+import org.apache.nifi.web.api.dto.WritablePermission;
 
 import javax.xml.bind.annotation.XmlRootElement;
 
-/**
- * A serialized representation of this class can be placed in the entity body of a response to the API.
- * This particular entity holds a reference to component that references a variable.
- */
-@XmlRootElement(name = "affectComponentEntity")
-public class AffectedComponentEntity extends ComponentEntity implements Permissible<AffectedComponentDTO> {
+@XmlRootElement(name = "parameterEntity")
+public class ParameterEntity extends Entity implements WritablePermission {
+    private Boolean canWrite;
+    private ParameterDTO parameter;
+
+    @ApiModelProperty("The parameter information")
+    public ParameterDTO getParameter() {
+        return parameter;
+    }
 
-    private AffectedComponentDTO component;
+    public void setParameter(final ParameterDTO parameter) {
+        this.parameter = parameter;
+    }
 
-    /**
-     * @return variable referencing components that is being serialized
-     */
     @Override
-    public AffectedComponentDTO getComponent() {
-        return component;
+    @ApiModelProperty(value = "Indicates whether the user can write a given resource.", readOnly = true)
+    public Boolean getCanWrite() {
+        return canWrite;
     }
 
     @Override
-    public void setComponent(AffectedComponentDTO component) {
-        this.component = component;
+    public void setCanWrite(final Boolean canWrite) {
+        this.canWrite = canWrite;
     }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileAccessPolicyProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileAccessPolicyProvider.java
index 3174e34..d29a669 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileAccessPolicyProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileAccessPolicyProvider.java
@@ -16,39 +16,6 @@
  */
 package org.apache.nifi.authorization;
 
-import java.io.ByteArrayInputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.StringWriter;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-import javax.xml.XMLConstants;
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBElement;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Marshaller;
-import javax.xml.bind.Unmarshaller;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-import javax.xml.stream.XMLOutputFactory;
-import javax.xml.stream.XMLStreamException;
-import javax.xml.stream.XMLStreamReader;
-import javax.xml.stream.XMLStreamWriter;
-import javax.xml.transform.stream.StreamSource;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.annotation.AuthorizerContext;
 import org.apache.nifi.authorization.exception.AuthorizationAccessException;
@@ -76,6 +43,40 @@ import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 import org.xml.sax.SAXException;
 
+import javax.xml.XMLConstants;
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBElement;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Marshaller;
+import javax.xml.bind.Unmarshaller;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.stream.XMLOutputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.XMLStreamReader;
+import javax.xml.stream.XMLStreamWriter;
+import javax.xml.transform.stream.StreamSource;
+import javax.xml.validation.Schema;
+import javax.xml.validation.SchemaFactory;
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
 public class FileAccessPolicyProvider implements ConfigurableAccessPolicyProvider {
 
     private static final Logger logger = LoggerFactory.getLogger(FileAccessPolicyProvider.class);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileUserGroupProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileUserGroupProvider.java
index d2066a9..5ff2d49 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileUserGroupProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileUserGroupProvider.java
@@ -93,7 +93,7 @@ public class FileUserGroupProvider implements ConfigurableUserGroupProvider {
         try {
             return JAXBContext.newInstance(contextPath, FileAuthorizer.class.getClassLoader());
         } catch (JAXBException e) {
-            throw new RuntimeException("Unable to create JAXBContext.");
+            throw new RuntimeException("Unable to create JAXBContext.", e);
         }
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAccessPolicyProviderTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAccessPolicyProviderTest.java
index 9d14af3..2f50051 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAccessPolicyProviderTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAccessPolicyProviderTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.authorization;
 
+import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
 import org.apache.nifi.authorization.exception.AuthorizerCreationException;
 import org.apache.nifi.authorization.resource.ResourceFactory;
@@ -186,11 +187,13 @@ public class FileAccessPolicyProviderTest {
 
         // this same configuration is being used for both the user group provider and the access policy provider
         configurationContext = mock(AuthorizerConfigurationContext.class);
-        when(configurationContext.getProperty(eq(FileAccessPolicyProvider.PROP_AUTHORIZATIONS_FILE))).thenReturn(new StandardPropertyValue(primaryAuthorizations.getPath(), null));
-        when(configurationContext.getProperty(eq(FileUserGroupProvider.PROP_TENANTS_FILE))).thenReturn(new StandardPropertyValue(primaryTenants.getPath(), null));
-        when(configurationContext.getProperty(eq(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY))).thenReturn(new StandardPropertyValue(null, null));
-        when(configurationContext.getProperty(eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE))).thenReturn(new StandardPropertyValue(null, null));
-        when(configurationContext.getProperty(eq(FileAccessPolicyProvider.PROP_USER_GROUP_PROVIDER))).thenReturn(new StandardPropertyValue("user-group-provider", null));
+        when(configurationContext.getProperty(eq(FileAccessPolicyProvider.PROP_AUTHORIZATIONS_FILE))).thenReturn(new StandardPropertyValue(primaryAuthorizations.getPath(), null,
+            ParameterLookup.EMPTY));
+        when(configurationContext.getProperty(eq(FileUserGroupProvider.PROP_TENANTS_FILE))).thenReturn(new StandardPropertyValue(primaryTenants.getPath(), null, ParameterLookup.EMPTY));
+        when(configurationContext.getProperty(eq(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY))).thenReturn(new StandardPropertyValue(null, null, ParameterLookup.EMPTY));
+        when(configurationContext.getProperty(eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE))).thenReturn(new StandardPropertyValue(null, null, ParameterLookup.EMPTY));
+        when(configurationContext.getProperty(eq(FileAccessPolicyProvider.PROP_USER_GROUP_PROVIDER))).thenReturn(new StandardPropertyValue("user-group-provider", null,
+            ParameterLookup.EMPTY));
         when(configurationContext.getProperties()).then((invocation) -> {
             final Map<String, String> properties = new HashMap<>();
 
@@ -275,7 +278,7 @@ public class FileAccessPolicyProviderTest {
     @Test
     public void testOnConfiguredWhenLegacyUsersFileProvidedWithOverlappingRoles() throws Exception {
         when(configurationContext.getProperty(eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)))
-                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users-multirole.xml", null));
+                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users-multirole.xml", null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@@ -298,7 +301,7 @@ public class FileAccessPolicyProviderTest {
         when(properties.getFlowConfigurationFile()).thenReturn(flowNoPorts);
 
         when(configurationContext.getProperty(eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)))
-                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users.xml", null));
+                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users.xml", null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@@ -320,7 +323,7 @@ public class FileAccessPolicyProviderTest {
     @Test
     public void testOnConfiguredWhenLegacyUsersFileProvided() throws Exception {
         when(configurationContext.getProperty(eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)))
-                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users.xml", null));
+                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users.xml", null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@@ -467,7 +470,7 @@ public class FileAccessPolicyProviderTest {
         accessPolicyProvider.setNiFiProperties(properties);
 
         when(configurationContext.getProperty(eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)))
-                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users-with-dns.xml", null));
+                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users-with-dns.xml", null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@@ -502,7 +505,7 @@ public class FileAccessPolicyProviderTest {
     @Test(expected = AuthorizerCreationException.class)
     public void testOnConfiguredWhenBadLegacyUsersFileProvided() throws Exception {
         when(configurationContext.getProperty(eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)))
-                .thenReturn(new StandardPropertyValue("src/test/resources/does-not-exist.xml", null));
+                .thenReturn(new StandardPropertyValue("src/test/resources/does-not-exist.xml", null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@@ -513,10 +516,10 @@ public class FileAccessPolicyProviderTest {
     public void testOnConfiguredWhenInitialAdminAndLegacyUsersProvided() throws Exception {
         final String adminIdentity = "admin-user";
         when(configurationContext.getProperty(eq(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY)))
-                .thenReturn(new StandardPropertyValue(adminIdentity, null));
+                .thenReturn(new StandardPropertyValue(adminIdentity, null, ParameterLookup.EMPTY));
 
         when(configurationContext.getProperty(eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)))
-                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users.xml", null));
+                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users.xml", null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@@ -540,7 +543,7 @@ public class FileAccessPolicyProviderTest {
         final String adminIdentity = "admin-user";
 
         when(configurationContext.getProperty(eq(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY)))
-                .thenReturn(new StandardPropertyValue(adminIdentity, null));
+                .thenReturn(new StandardPropertyValue(adminIdentity, null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@@ -580,7 +583,7 @@ public class FileAccessPolicyProviderTest {
 
         final String adminIdentity = "admin-user";
         when(configurationContext.getProperty(eq(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY)))
-                .thenReturn(new StandardPropertyValue(adminIdentity, null));
+                .thenReturn(new StandardPropertyValue(adminIdentity, null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@@ -620,7 +623,7 @@ public class FileAccessPolicyProviderTest {
 
         final String adminIdentity = "admin-user";
         when(configurationContext.getProperty(eq(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY)))
-                .thenReturn(new StandardPropertyValue(adminIdentity, null));
+                .thenReturn(new StandardPropertyValue(adminIdentity, null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@@ -663,7 +666,7 @@ public class FileAccessPolicyProviderTest {
 
         final String adminIdentity = "CN=localhost, OU=Apache NiFi, O=Apache, L=Santa Monica, ST=CA, C=US";
         when(configurationContext.getProperty(eq(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY)))
-                .thenReturn(new StandardPropertyValue(adminIdentity, null));
+                .thenReturn(new StandardPropertyValue(adminIdentity, null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@@ -683,18 +686,18 @@ public class FileAccessPolicyProviderTest {
         final String nodeIdentity2 = "node2";
 
         when(configurationContext.getProperty(eq(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY)))
-                .thenReturn(new StandardPropertyValue(adminIdentity, null));
+                .thenReturn(new StandardPropertyValue(adminIdentity, null, ParameterLookup.EMPTY));
         when(configurationContext.getProperty(eq(FileAccessPolicyProvider.PROP_NODE_IDENTITY_PREFIX + "1")))
-                .thenReturn(new StandardPropertyValue(nodeIdentity1, null));
+                .thenReturn(new StandardPropertyValue(nodeIdentity1, null, ParameterLookup.EMPTY));
         when(configurationContext.getProperty(eq(FileAccessPolicyProvider.PROP_NODE_IDENTITY_PREFIX + "2")))
-                .thenReturn(new StandardPropertyValue(nodeIdentity2, null));
+                .thenReturn(new StandardPropertyValue(nodeIdentity2, null, ParameterLookup.EMPTY));
 
         when(configurationContext.getProperty(eq(FileUserGroupProvider.PROP_INITIAL_USER_IDENTITY_PREFIX + "1")))
-                .thenReturn(new StandardPropertyValue(adminIdentity, null));
+                .thenReturn(new StandardPropertyValue(adminIdentity, null, ParameterLookup.EMPTY));
         when(configurationContext.getProperty(eq(FileUserGroupProvider.PROP_INITIAL_USER_IDENTITY_PREFIX + "2")))
-                .thenReturn(new StandardPropertyValue(nodeIdentity1, null));
+                .thenReturn(new StandardPropertyValue(nodeIdentity1, null, ParameterLookup.EMPTY));
         when(configurationContext.getProperty(eq(FileUserGroupProvider.PROP_INITIAL_USER_IDENTITY_PREFIX + "3")))
-                .thenReturn(new StandardPropertyValue(nodeIdentity2, null));
+                .thenReturn(new StandardPropertyValue(nodeIdentity2, null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@@ -719,18 +722,18 @@ public class FileAccessPolicyProviderTest {
         final String nodeIdentity2 = "node2";
 
         when(configurationContext.getProperty(Mockito.eq(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY)))
-                .thenReturn(new StandardPropertyValue(adminIdentity, null));
+                .thenReturn(new StandardPropertyValue(adminIdentity, null, ParameterLookup.EMPTY));
         when(configurationContext.getProperty(Mockito.eq(FileAccessPolicyProvider.PROP_NODE_IDENTITY_PREFIX + "1")))
-                .thenReturn(new StandardPropertyValue(nodeIdentity1, null));
+                .thenReturn(new StandardPropertyValue(nodeIdentity1, null, ParameterLookup.EMPTY));
         when(configurationContext.getProperty(Mockito.eq(FileAccessPolicyProvider.PROP_NODE_IDENTITY_PREFIX + "2")))
-                .thenReturn(new StandardPropertyValue(nodeIdentity2, null));
+                .thenReturn(new StandardPropertyValue(nodeIdentity2, null, ParameterLookup.EMPTY));
 
         when(configurationContext.getProperty(eq(FileUserGroupProvider.PROP_INITIAL_USER_IDENTITY_PREFIX + "1")))
-                .thenReturn(new StandardPropertyValue(adminIdentity, null));
+                .thenReturn(new StandardPropertyValue(adminIdentity, null, ParameterLookup.EMPTY));
         when(configurationContext.getProperty(eq(FileUserGroupProvider.PROP_INITIAL_USER_IDENTITY_PREFIX + "2")))
-                .thenReturn(new StandardPropertyValue(nodeIdentity1, null));
+                .thenReturn(new StandardPropertyValue(nodeIdentity1, null, ParameterLookup.EMPTY));
         when(configurationContext.getProperty(eq(FileUserGroupProvider.PROP_INITIAL_USER_IDENTITY_PREFIX + "3")))
-                .thenReturn(new StandardPropertyValue(nodeIdentity2, null));
+                .thenReturn(new StandardPropertyValue(nodeIdentity2, null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, TENANTS_FOR_ADMIN_AND_NODES);
@@ -757,9 +760,9 @@ public class FileAccessPolicyProviderTest {
         final String nodeIdentity2 = "node2";
 
         when(configurationContext.getProperty(eq(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY)))
-                .thenReturn(new StandardPropertyValue(adminIdentity, null));
+                .thenReturn(new StandardPropertyValue(adminIdentity, null, ParameterLookup.EMPTY));
         when(configurationContext.getProperty(eq(FileAccessPolicyProvider.PROP_NODE_GROUP_NAME)))
-                .thenReturn(new StandardPropertyValue(nodeGroupName, null));
+                .thenReturn(new StandardPropertyValue(nodeGroupName, null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, TENANTS_FOR_ADMIN_AND_NODE_GROUP);
@@ -782,9 +785,9 @@ public class FileAccessPolicyProviderTest {
         final String nodeGroupIdentifier = "cluster-nodes";
 
         when(configurationContext.getProperty(eq(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY)))
-            .thenReturn(new StandardPropertyValue(adminIdentity, null));
+            .thenReturn(new StandardPropertyValue(adminIdentity, null, ParameterLookup.EMPTY));
         when(configurationContext.getProperty(eq(FileAccessPolicyProvider.PROP_NODE_GROUP_NAME)))
-            .thenReturn(new StandardPropertyValue("", null));
+            .thenReturn(new StandardPropertyValue("", null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, TENANTS_FOR_ADMIN_AND_NODE_GROUP);
@@ -800,9 +803,9 @@ public class FileAccessPolicyProviderTest {
         final String adminIdentity = "admin-user";
 
         when(configurationContext.getProperty(eq(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY)))
-            .thenReturn(new StandardPropertyValue(adminIdentity, null));
+            .thenReturn(new StandardPropertyValue(adminIdentity, null, ParameterLookup.EMPTY));
         when(configurationContext.getProperty(eq(FileAccessPolicyProvider.PROP_NODE_GROUP_NAME)))
-            .thenReturn(new StandardPropertyValue("nonexistent", null));
+            .thenReturn(new StandardPropertyValue("nonexistent", null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, TENANTS_FOR_ADMIN_AND_NODE_GROUP);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java
index a1d477c..e83e97a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.authorization;
 
+import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
 import org.apache.nifi.authorization.AuthorizationResult.Result;
 import org.apache.nifi.authorization.exception.AuthorizerCreationException;
@@ -189,8 +190,9 @@ public class FileAuthorizerTest {
         when(properties.getFlowConfigurationFile()).thenReturn(flow);
 
         configurationContext = mock(AuthorizerConfigurationContext.class);
-        when(configurationContext.getProperty(Mockito.eq(FileAccessPolicyProvider.PROP_AUTHORIZATIONS_FILE))).thenReturn(new StandardPropertyValue(primaryAuthorizations.getPath(), null));
-        when(configurationContext.getProperty(Mockito.eq(FileUserGroupProvider.PROP_TENANTS_FILE))).thenReturn(new StandardPropertyValue(primaryTenants.getPath(), null));
+        when(configurationContext.getProperty(Mockito.eq(FileAccessPolicyProvider.PROP_AUTHORIZATIONS_FILE))).thenReturn(new StandardPropertyValue(primaryAuthorizations.getPath(), null,
+            ParameterLookup.EMPTY));
+        when(configurationContext.getProperty(Mockito.eq(FileUserGroupProvider.PROP_TENANTS_FILE))).thenReturn(new StandardPropertyValue(primaryTenants.getPath(), null, ParameterLookup.EMPTY));
         when(configurationContext.getProperties()).then((invocation) -> {
             final Map<String, String> properties = new HashMap<>();
 
@@ -244,7 +246,7 @@ public class FileAuthorizerTest {
     @Test
     public void testOnConfiguredWhenLegacyUsersFileProvidedWithOverlappingRoles() throws Exception {
         when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)))
-                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users-multirole.xml", null));
+                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users-multirole.xml", null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@@ -269,7 +271,7 @@ public class FileAuthorizerTest {
         when(properties.getFlowConfigurationFile()).thenReturn(flowNoPorts);
 
         when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)))
-                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users.xml", null));
+                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users.xml", null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@@ -289,7 +291,7 @@ public class FileAuthorizerTest {
     @Test
     public void testOnConfiguredWhenLegacyUsersFileProvided() throws Exception {
         when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)))
-                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users.xml", null));
+                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users.xml", null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@@ -453,7 +455,7 @@ public class FileAuthorizerTest {
         authorizer.setNiFiProperties(properties);
 
         when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)))
-                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users-with-dns.xml", null));
+                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users-with-dns.xml", null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@@ -503,7 +505,7 @@ public class FileAuthorizerTest {
     @Test(expected = AuthorizerCreationException.class)
     public void testOnConfiguredWhenBadLegacyUsersFileProvided() throws Exception {
         when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)))
-                .thenReturn(new StandardPropertyValue("src/test/resources/does-not-exist.xml", null));
+                .thenReturn(new StandardPropertyValue("src/test/resources/does-not-exist.xml", null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@@ -514,10 +516,10 @@ public class FileAuthorizerTest {
     public void testOnConfiguredWhenInitialAdminAndLegacyUsersProvided() throws Exception {
         final String adminIdentity = "admin-user";
         when(configurationContext.getProperty(Mockito.eq(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY)))
-                .thenReturn(new StandardPropertyValue(adminIdentity, null));
+                .thenReturn(new StandardPropertyValue(adminIdentity, null, ParameterLookup.EMPTY));
 
         when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)))
-                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users.xml", null));
+                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users.xml", null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@@ -542,7 +544,7 @@ public class FileAuthorizerTest {
         final String adminIdentity = "admin-user";
 
         when(configurationContext.getProperty(Mockito.eq(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY)))
-                .thenReturn(new StandardPropertyValue(adminIdentity, null));
+                .thenReturn(new StandardPropertyValue(adminIdentity, null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@@ -580,7 +582,7 @@ public class FileAuthorizerTest {
 
         final String adminIdentity = "admin-user";
         when(configurationContext.getProperty(Mockito.eq(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY)))
-                .thenReturn(new StandardPropertyValue(adminIdentity, null));
+                .thenReturn(new StandardPropertyValue(adminIdentity, null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@@ -618,7 +620,7 @@ public class FileAuthorizerTest {
 
         final String adminIdentity = "admin-user";
         when(configurationContext.getProperty(Mockito.eq(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY)))
-                .thenReturn(new StandardPropertyValue(adminIdentity, null));
+                .thenReturn(new StandardPropertyValue(adminIdentity, null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@@ -659,7 +661,7 @@ public class FileAuthorizerTest {
 
         final String adminIdentity = "CN=localhost, OU=Apache NiFi, O=Apache, L=Santa Monica, ST=CA, C=US";
         when(configurationContext.getProperty(Mockito.eq(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY)))
-                .thenReturn(new StandardPropertyValue(adminIdentity, null));
+                .thenReturn(new StandardPropertyValue(adminIdentity, null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@@ -679,11 +681,11 @@ public class FileAuthorizerTest {
         final String nodeIdentity2 = "node2";
 
         when(configurationContext.getProperty(Mockito.eq(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY)))
-                .thenReturn(new StandardPropertyValue(adminIdentity, null));
+                .thenReturn(new StandardPropertyValue(adminIdentity, null, ParameterLookup.EMPTY));
         when(configurationContext.getProperty(Mockito.eq(FileAccessPolicyProvider.PROP_NODE_IDENTITY_PREFIX + "1")))
-                .thenReturn(new StandardPropertyValue(nodeIdentity1, null));
+                .thenReturn(new StandardPropertyValue(nodeIdentity1, null, ParameterLookup.EMPTY));
         when(configurationContext.getProperty(Mockito.eq(FileAccessPolicyProvider.PROP_NODE_IDENTITY_PREFIX + "2")))
-                .thenReturn(new StandardPropertyValue(nodeIdentity2, null));
+                .thenReturn(new StandardPropertyValue(nodeIdentity2, null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@@ -712,11 +714,11 @@ public class FileAuthorizerTest {
         final String nodeIdentity2 = "node2";
 
         when(configurationContext.getProperty(Mockito.eq(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY)))
-                .thenReturn(new StandardPropertyValue(adminIdentity, null));
+                .thenReturn(new StandardPropertyValue(adminIdentity, null, ParameterLookup.EMPTY));
         when(configurationContext.getProperty(Mockito.eq(FileAccessPolicyProvider.PROP_NODE_IDENTITY_PREFIX + "1")))
-                .thenReturn(new StandardPropertyValue(nodeIdentity1, null));
+                .thenReturn(new StandardPropertyValue(nodeIdentity1, null, ParameterLookup.EMPTY));
         when(configurationContext.getProperty(Mockito.eq(FileAccessPolicyProvider.PROP_NODE_IDENTITY_PREFIX + "2")))
-                .thenReturn(new StandardPropertyValue(nodeIdentity2, null));
+                .thenReturn(new StandardPropertyValue(nodeIdentity2, null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, TENANTS_FOR_ADMIN_AND_NODES);
@@ -756,11 +758,11 @@ public class FileAuthorizerTest {
         final String nodeIdentity2 = "CN=node2, OU=Apache NiFi, O=Apache, L=Santa Monica, ST=CA, C=US";
 
         when(configurationContext.getProperty(Mockito.eq(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY)))
-                .thenReturn(new StandardPropertyValue(adminIdentity, null));
+                .thenReturn(new StandardPropertyValue(adminIdentity, null, ParameterLookup.EMPTY));
         when(configurationContext.getProperty(Mockito.eq(FileAccessPolicyProvider.PROP_NODE_IDENTITY_PREFIX + "1")))
-                .thenReturn(new StandardPropertyValue(nodeIdentity1, null));
+                .thenReturn(new StandardPropertyValue(nodeIdentity1, null, ParameterLookup.EMPTY));
         when(configurationContext.getProperty(Mockito.eq(FileAccessPolicyProvider.PROP_NODE_IDENTITY_PREFIX + "2")))
-                .thenReturn(new StandardPropertyValue(nodeIdentity2, null));
+                .thenReturn(new StandardPropertyValue(nodeIdentity2, null, ParameterLookup.EMPTY));
 
         writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileUserGroupProviderTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileUserGroupProviderTest.java
index 8be051e..71d376d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileUserGroupProviderTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileUserGroupProviderTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.authorization;
 
+import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
 import org.apache.nifi.authorization.exception.AuthorizerCreationException;
 import org.apache.nifi.components.PropertyValue;
@@ -109,8 +110,8 @@ public class FileUserGroupProviderTest {
         when(properties.getRestoreDirectory()).thenReturn(restoreTenants.getParentFile());
 
         configurationContext = mock(AuthorizerConfigurationContext.class);
-        when(configurationContext.getProperty(eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE))).thenReturn(new StandardPropertyValue(null, null));
-        when(configurationContext.getProperty(eq(FileUserGroupProvider.PROP_TENANTS_FILE))).thenReturn(new StandardPropertyValue(primaryTenants.getPath(), null));
+        when(configurationContext.getProperty(eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE))).thenReturn(new StandardPropertyValue(null, null, ParameterLookup.EMPTY));
+        when(configurationContext.getProperty(eq(FileUserGroupProvider.PROP_TENANTS_FILE))).thenReturn(new StandardPropertyValue(primaryTenants.getPath(), null, ParameterLookup.EMPTY));
         when(configurationContext.getProperties()).then((invocation) -> {
             final Map<String, String> properties = new HashMap<>();
 
@@ -152,7 +153,7 @@ public class FileUserGroupProviderTest {
     @Test
     public void testOnConfiguredWhenLegacyUsersFileProvided() throws Exception {
         when(configurationContext.getProperty(eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)))
-                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users.xml", null));
+                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users.xml", null, ParameterLookup.EMPTY));
 
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
         userGroupProvider.onConfigured(configurationContext);
@@ -197,7 +198,7 @@ public class FileUserGroupProviderTest {
         userGroupProvider.setNiFiProperties(properties);
 
         when(configurationContext.getProperty(eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)))
-                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users-with-dns.xml", null));
+                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users-with-dns.xml", null, ParameterLookup.EMPTY));
 
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
         userGroupProvider.onConfigured(configurationContext);
@@ -243,7 +244,7 @@ public class FileUserGroupProviderTest {
         userGroupProvider.setNiFiProperties(properties);
 
         when(configurationContext.getProperty(eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)))
-                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users-with-dns.xml", null));
+                .thenReturn(new StandardPropertyValue("src/test/resources/authorized-users-with-dns.xml", null, ParameterLookup.EMPTY));
 
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
         userGroupProvider.onConfigured(configurationContext);
@@ -276,7 +277,7 @@ public class FileUserGroupProviderTest {
     @Test(expected = AuthorizerCreationException.class)
     public void testOnConfiguredWhenBadLegacyUsersFileProvided() throws Exception {
         when(configurationContext.getProperty(eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)))
-                .thenReturn(new StandardPropertyValue("src/test/resources/does-not-exist.xml", null));
+                .thenReturn(new StandardPropertyValue("src/test/resources/does-not-exist.xml", null, ParameterLookup.EMPTY));
 
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
         userGroupProvider.onConfigured(configurationContext);
@@ -298,11 +299,11 @@ public class FileUserGroupProviderTest {
         final String nodeIdentity2 = "node-identity-2";
 
         when(configurationContext.getProperty(eq(FileUserGroupProvider.PROP_INITIAL_USER_IDENTITY_PREFIX + "1")))
-                .thenReturn(new StandardPropertyValue(adminIdentity, null));
+                .thenReturn(new StandardPropertyValue(adminIdentity, null, ParameterLookup.EMPTY));
         when(configurationContext.getProperty(eq(FileUserGroupProvider.PROP_INITIAL_USER_IDENTITY_PREFIX + "2")))
-                .thenReturn(new StandardPropertyValue(nodeIdentity1, null));
+                .thenReturn(new StandardPropertyValue(nodeIdentity1, null, ParameterLookup.EMPTY));
         when(configurationContext.getProperty(eq(FileUserGroupProvider.PROP_INITIAL_USER_IDENTITY_PREFIX + "3")))
-                .thenReturn(new StandardPropertyValue(nodeIdentity2, null));
+                .thenReturn(new StandardPropertyValue(nodeIdentity2, null, ParameterLookup.EMPTY));
 
         writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
         userGroupProvider.onConfigured(configurationContext);
@@ -323,11 +324,11 @@ public class FileUserGroupProviderTest {
 
         // despite setting initial users, they will not be loaded as the tenants file is non-empty
         when(configurationContext.getProperty(eq(FileUserGroupProvider.PROP_INITIAL_USER_IDENTITY_PREFIX + "1")))
-                .thenReturn(new StandardPropertyValue(adminIdentity, null));
+                .thenReturn(new StandardPropertyValue(adminIdentity, null, ParameterLookup.EMPTY));
         when(configurationContext.getProperty(eq(FileUserGroupProvider.PROP_INITIAL_USER_IDENTITY_PREFIX + "2")))
-                .thenReturn(new StandardPropertyValue(nodeIdentity1, null));
+                .thenReturn(new StandardPropertyValue(nodeIdentity1, null, ParameterLookup.EMPTY));
         when(configurationContext.getProperty(eq(FileUserGroupProvider.PROP_INITIAL_USER_IDENTITY_PREFIX + "3")))
-                .thenReturn(new StandardPropertyValue(nodeIdentity2, null));
+                .thenReturn(new StandardPropertyValue(nodeIdentity2, null, ParameterLookup.EMPTY));
 
         writeFile(primaryTenants, SIMPLE_TENANTS_BY_USER);
         userGroupProvider.onConfigured(configurationContext);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/StandardAuthorizerConfigurationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/StandardAuthorizerConfigurationContext.java
index 3010c92..a1a2511 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/StandardAuthorizerConfigurationContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/StandardAuthorizerConfigurationContext.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.authorization;
 
+import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
 import org.apache.nifi.components.PropertyValue;
 
@@ -48,7 +49,7 @@ public class StandardAuthorizerConfigurationContext implements AuthorizerConfigu
 
     @Override
     public PropertyValue getProperty(String property) {
-        return new StandardPropertyValue(properties.get(property), null);
+        return new StandardPropertyValue(properties.get(property), null, ParameterLookup.EMPTY);
     }
 
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/resource/ResourceFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/resource/ResourceFactory.java
index 9d3b532..386f90a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/resource/ResourceFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/resource/ResourceFactory.java
@@ -262,6 +262,25 @@ public final class ResourceFactory {
         }
     };
 
+    private final static Resource PARAMETER_CONTEXTS_RESOURCE = new Resource() {
+
+        @Override
+        public String getIdentifier() {
+            return ResourceType.ParameterContext.getValue();
+        }
+
+        @Override
+        public String getName() {
+            return "Parameter Contexts";
+        }
+
+        @Override
+        public String getSafeDescription() {
+            return "parameter contexts";
+        }
+    };
+
+
     /**
      * Gets the Resource for accessing the Controller. This includes Controller level configuration, bulletins, reporting tasks, and the cluster.
      *
@@ -290,6 +309,14 @@ public final class ResourceFactory {
     }
 
     /**
+     * Gets the Resource for accessing the Parameter Contexts.
+     * @return The resource for access Parameter Contexts
+     */
+    public static Resource getParameterContextsResource() {
+        return PARAMETER_CONTEXTS_RESOURCE;
+    }
+
+    /**
      * Gets the Resource for accessing provenance. Access to this Resource allows the user to access data provenance. However, additional authorization
      * is required based on the component that generated the event and the attributes of the event.
      *
@@ -525,6 +552,9 @@ public final class ResourceFactory {
                     case Label:
                         componentType = "Label";
                         break;
+                    case ParameterContext:
+                        componentType = "Parameter Context";
+                        break;
                     default:
                         componentType = "Component";
                         break;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/resource/ResourceType.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/resource/ResourceType.java
index a24b904..45582fc 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/resource/ResourceType.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/resource/ResourceType.java
@@ -41,7 +41,8 @@ public enum ResourceType {
     RestrictedComponents("/restricted-components"),
     Operation("/operation"),
     Template("/templates"),
-    Tenant("/tenants");
+    Tenant("/tenants"),
+    ParameterContext("/parameter-contexts");
 
     final String value;
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/test/java/org/apache/nifi/authorization/CompositeConfigurableUserGroupProviderTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/test/java/org/apache/nifi/authorization/CompositeConfigurableUserGroupProviderTest.java
index bb4f988..aa96bee 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/test/java/org/apache/nifi/authorization/CompositeConfigurableUserGroupProviderTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/test/java/org/apache/nifi/authorization/CompositeConfigurableUserGroupProviderTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.authorization;
 
+import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
 import org.apache.nifi.authorization.exception.AuthorizerCreationException;
 import org.junit.Test;
@@ -47,14 +48,14 @@ public class CompositeConfigurableUserGroupProviderTest extends CompositeUserGro
     @Test(expected = AuthorizerCreationException.class)
     public void testNoConfigurableUserGroupProviderSpecified() throws Exception {
         initCompositeUserGroupProvider(new CompositeConfigurableUserGroupProvider(), null, configurationContext -> {
-            when(configurationContext.getProperty(PROP_CONFIGURABLE_USER_GROUP_PROVIDER)).thenReturn(new StandardPropertyValue(null, null));
+            when(configurationContext.getProperty(PROP_CONFIGURABLE_USER_GROUP_PROVIDER)).thenReturn(new StandardPropertyValue(null, null, ParameterLookup.EMPTY));
         });
     }
 
     @Test(expected = AuthorizerCreationException.class)
     public void testUnknownConfigurableUserGroupProvider() throws Exception {
         initCompositeUserGroupProvider(new CompositeConfigurableUserGroupProvider(), null, configurationContext -> {
-            when(configurationContext.getProperty(PROP_CONFIGURABLE_USER_GROUP_PROVIDER)).thenReturn(new StandardPropertyValue("unknown-user-group-provider", null));
+            when(configurationContext.getProperty(PROP_CONFIGURABLE_USER_GROUP_PROVIDER)).thenReturn(new StandardPropertyValue("unknown-user-group-provider", null, ParameterLookup.EMPTY));
         });
     }
 
@@ -63,7 +64,7 @@ public class CompositeConfigurableUserGroupProviderTest extends CompositeUserGro
         initCompositeUserGroupProvider(new CompositeConfigurableUserGroupProvider(), lookup -> {
             when(lookup.getUserGroupProvider(eq(NOT_CONFIGURABLE_USER_GROUP_PROVIDER))).thenReturn(mock(UserGroupProvider.class));
         }, configurationContext -> {
-            when(configurationContext.getProperty(PROP_CONFIGURABLE_USER_GROUP_PROVIDER)).thenReturn(new StandardPropertyValue(NOT_CONFIGURABLE_USER_GROUP_PROVIDER, null));
+            when(configurationContext.getProperty(PROP_CONFIGURABLE_USER_GROUP_PROVIDER)).thenReturn(new StandardPropertyValue(NOT_CONFIGURABLE_USER_GROUP_PROVIDER, null, ParameterLookup.EMPTY));
         });
     }
 
@@ -81,7 +82,7 @@ public class CompositeConfigurableUserGroupProviderTest extends CompositeUserGro
 
         // Mock AuthorizerConfigurationContext to introduce the duplicate provider ids
         final AuthorizerConfigurationContext configurationContext = mock(AuthorizerConfigurationContext.class);
-        when(configurationContext.getProperty(PROP_CONFIGURABLE_USER_GROUP_PROVIDER)).thenReturn(new StandardPropertyValue(CONFIGURABLE_USER_GROUP_PROVIDER, null));
+        when(configurationContext.getProperty(PROP_CONFIGURABLE_USER_GROUP_PROVIDER)).thenReturn(new StandardPropertyValue(CONFIGURABLE_USER_GROUP_PROVIDER, null, ParameterLookup.EMPTY));
         Map<String, String> configurationContextProperties = new HashMap<>();
         configurationContextProperties.put(PROP_USER_GROUP_PROVIDER_PREFIX + "1", CONFIGURABLE_USER_GROUP_PROVIDER);
         configurationContextProperties.put(PROP_USER_GROUP_PROVIDER_PREFIX + "2", NOT_CONFIGURABLE_USER_GROUP_PROVIDER);
@@ -98,7 +99,7 @@ public class CompositeConfigurableUserGroupProviderTest extends CompositeUserGro
         final UserGroupProvider userGroupProvider = initCompositeUserGroupProvider(new CompositeConfigurableUserGroupProvider(), lookup -> {
             when(lookup.getUserGroupProvider(eq(CONFIGURABLE_USER_GROUP_PROVIDER))).thenReturn(getConfigurableUserGroupProvider());
         }, configurationContext -> {
-            when(configurationContext.getProperty(PROP_CONFIGURABLE_USER_GROUP_PROVIDER)).thenReturn(new StandardPropertyValue(CONFIGURABLE_USER_GROUP_PROVIDER, null));
+            when(configurationContext.getProperty(PROP_CONFIGURABLE_USER_GROUP_PROVIDER)).thenReturn(new StandardPropertyValue(CONFIGURABLE_USER_GROUP_PROVIDER, null, ParameterLookup.EMPTY));
         });
 
         // users and groups
@@ -123,7 +124,7 @@ public class CompositeConfigurableUserGroupProviderTest extends CompositeUserGro
         final UserGroupProvider userGroupProvider = initCompositeUserGroupProvider(new CompositeConfigurableUserGroupProvider(), lookup -> {
             when(lookup.getUserGroupProvider(eq(CONFIGURABLE_USER_GROUP_PROVIDER))).thenReturn(getConfigurableUserGroupProvider());
         }, configurationContext -> {
-            when(configurationContext.getProperty(PROP_CONFIGURABLE_USER_GROUP_PROVIDER)).thenReturn(new StandardPropertyValue(CONFIGURABLE_USER_GROUP_PROVIDER, null));
+            when(configurationContext.getProperty(PROP_CONFIGURABLE_USER_GROUP_PROVIDER)).thenReturn(new StandardPropertyValue(CONFIGURABLE_USER_GROUP_PROVIDER, null, ParameterLookup.EMPTY));
         }, getConflictingUserGroupProvider());
 
         // users and groups
@@ -162,7 +163,7 @@ public class CompositeConfigurableUserGroupProviderTest extends CompositeUserGro
         final UserGroupProvider userGroupProvider = initCompositeUserGroupProvider(new CompositeConfigurableUserGroupProvider(), lookup -> {
             when(lookup.getUserGroupProvider(eq(CONFIGURABLE_USER_GROUP_PROVIDER))).thenReturn(getConfigurableUserGroupProvider());
         }, configurationContext -> {
-            when(configurationContext.getProperty(PROP_CONFIGURABLE_USER_GROUP_PROVIDER)).thenReturn(new StandardPropertyValue(CONFIGURABLE_USER_GROUP_PROVIDER, null));
+            when(configurationContext.getProperty(PROP_CONFIGURABLE_USER_GROUP_PROVIDER)).thenReturn(new StandardPropertyValue(CONFIGURABLE_USER_GROUP_PROVIDER, null, ParameterLookup.EMPTY));
         }, getCollaboratingUserGroupProvider());
 
         // users and groups
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/test/java/org/apache/nifi/authorization/CompositeUserGroupProviderTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/test/java/org/apache/nifi/authorization/CompositeUserGroupProviderTest.java
index 032590d..674808f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/test/java/org/apache/nifi/authorization/CompositeUserGroupProviderTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/test/java/org/apache/nifi/authorization/CompositeUserGroupProviderTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.authorization;
 
+import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
 import org.apache.nifi.authorization.exception.AuthorizerCreationException;
 import org.junit.Test;
@@ -54,7 +55,7 @@ public class CompositeUserGroupProviderTest extends CompositeUserGroupProviderTe
 
         // configuration
         final AuthorizerConfigurationContext configurationContext = mock(AuthorizerConfigurationContext.class);
-        when(configurationContext.getProperty(eq(PROP_USER_GROUP_PROVIDER_PREFIX + "1"))).thenReturn(new StandardPropertyValue(String.valueOf("1"), null));
+        when(configurationContext.getProperty(eq(PROP_USER_GROUP_PROVIDER_PREFIX + "1"))).thenReturn(new StandardPropertyValue(String.valueOf("1"), null, ParameterLookup.EMPTY));
         mockProperties(configurationContext);
 
         final CompositeUserGroupProvider compositeUserGroupProvider = new CompositeUserGroupProvider();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/test/java/org/apache/nifi/authorization/CompositeUserGroupProviderTestBase.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/test/java/org/apache/nifi/authorization/CompositeUserGroupProviderTestBase.java
index a3f3bb2..c11e5e9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/test/java/org/apache/nifi/authorization/CompositeUserGroupProviderTestBase.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/test/java/org/apache/nifi/authorization/CompositeUserGroupProviderTestBase.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.authorization;
 
+import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
 import org.apache.nifi.components.PropertyValue;
 
@@ -220,7 +221,7 @@ public class CompositeUserGroupProviderTestBase {
         final AuthorizerConfigurationContext configurationContext = mock(AuthorizerConfigurationContext.class);
 
         for (int i = 1; i <= providers.length; i++) {
-            when(configurationContext.getProperty(eq(PROP_USER_GROUP_PROVIDER_PREFIX + i))).thenReturn(new StandardPropertyValue(String.valueOf(i), null));
+            when(configurationContext.getProperty(eq(PROP_USER_GROUP_PROVIDER_PREFIX + i))).thenReturn(new StandardPropertyValue(String.valueOf(i), null, ParameterLookup.EMPTY));
         }
 
         // allow callers to mock additional properties
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/test/java/org/apache/nifi/authorization/StandardManagedAuthorizerTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/test/java/org/apache/nifi/authorization/StandardManagedAuthorizerTest.java
index 7dd9d97..36c3724 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/test/java/org/apache/nifi/authorization/StandardManagedAuthorizerTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/test/java/org/apache/nifi/authorization/StandardManagedAuthorizerTest.java
@@ -16,7 +16,7 @@
  */
 package org.apache.nifi.authorization;
 
-
+import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
 import org.apache.nifi.authorization.exception.AuthorizationAccessException;
 import org.apache.nifi.authorization.exception.AuthorizerCreationException;
@@ -422,7 +422,7 @@ public class StandardManagedAuthorizerTest {
         final StandardManagedAuthorizer managedAuthorizer = new StandardManagedAuthorizer();
 
         final AuthorizerConfigurationContext configurationContext = mock(AuthorizerConfigurationContext.class);
-        when(configurationContext.getProperty("Access Policy Provider")).thenReturn(new StandardPropertyValue("access-policy-provider", null));
+        when(configurationContext.getProperty("Access Policy Provider")).thenReturn(new StandardPropertyValue("access-policy-provider", null, ParameterLookup.EMPTY));
 
         final AccessPolicyProviderLookup accessPolicyProviderLookup = mock(AccessPolicyProviderLookup.class);
         when(accessPolicyProviderLookup.getAccessPolicyProvider("access-policy-provider")).thenReturn(accessPolicyProvider);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java
index 0cd550b..78cd803 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java
@@ -44,6 +44,10 @@ import org.apache.nifi.cluster.coordination.http.endpoints.LabelEndpointMerger;
 import org.apache.nifi.cluster.coordination.http.endpoints.LabelsEndpointMerger;
 import org.apache.nifi.cluster.coordination.http.endpoints.ListFlowFilesEndpointMerger;
 import org.apache.nifi.cluster.coordination.http.endpoints.OutputPortsEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.ParameterContextEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.ParameterContextUpdateEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.ParameterContextValidationMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.ParameterContextsEndpointMerger;
 import org.apache.nifi.cluster.coordination.http.endpoints.PortEndpointMerger;
 import org.apache.nifi.cluster.coordination.http.endpoints.PortStatusEndpointMerger;
 import org.apache.nifi.cluster.coordination.http.endpoints.PrioritizerTypesEndpointMerger;
@@ -157,6 +161,10 @@ public class StandardHttpResponseMapper implements HttpResponseMapper {
         endpointMergers.add(new SearchUsersEndpointMerger());
         endpointMergers.add(new VariableRegistryEndpointMerger());
         endpointMergers.add(new ProcessorDiagnosticsEndpointMerger(snapshotMillis));
+        endpointMergers.add(new ParameterContextValidationMerger());
+        endpointMergers.add(new ParameterContextsEndpointMerger());
+        endpointMergers.add(new ParameterContextEndpointMerger());
+        endpointMergers.add(new ParameterContextUpdateEndpointMerger());
     }
 
     @Override
@@ -178,7 +186,7 @@ public class StandardHttpResponseMapper implements HttpResponseMapper {
         }
 
         // Determine which responses are successful
-        final Set<NodeResponse> successResponses = nodeResponses.stream().filter(p -> p.is2xx()).collect(Collectors.toSet());
+        final Set<NodeResponse> successResponses = nodeResponses.stream().filter(NodeResponse::is2xx).collect(Collectors.toSet());
         final Set<NodeResponse> problematicResponses = nodeResponses.stream().filter(p -> !p.is2xx()).collect(Collectors.toSet());
 
         final NodeResponse clientResponse;
@@ -192,7 +200,7 @@ public class StandardHttpResponseMapper implements HttpResponseMapper {
             clientResponse = successResponses.iterator().next();
         }
 
-        if (merge == false) {
+        if (!merge) {
             return clientResponse;
         }
 
@@ -215,7 +223,7 @@ public class StandardHttpResponseMapper implements HttpResponseMapper {
             return allResponses.stream().filter(p -> !p.is2xx()).collect(Collectors.toSet());
         } else {
             // If no node is successful, we consider a problematic response to be only those that are 5xx
-            return allResponses.stream().filter(p -> p.is5xx()).collect(Collectors.toSet());
+            return allResponses.stream().filter(NodeResponse::is5xx).collect(Collectors.toSet());
         }
     }
 
@@ -229,7 +237,7 @@ public class StandardHttpResponseMapper implements HttpResponseMapper {
     }
 
     private boolean hasSuccessfulResponse(final Set<NodeResponse> allResponses) {
-        return allResponses.stream().anyMatch(p -> p.is2xx());
+        return allResponses.stream().anyMatch(NodeResponse::is2xx);
     }
 
     private void drainResponses(final Set<NodeResponse> responses, final NodeResponse exclude) {
@@ -237,7 +245,7 @@ public class StandardHttpResponseMapper implements HttpResponseMapper {
                 .parallel() // "parallelize" the draining of the responses, since we have multiple streams to consume
                 .filter(response -> response != exclude) // don't include the explicitly excluded node
                 .filter(response -> response.getStatus() != RequestReplicator.NODE_CONTINUE_STATUS_CODE) // don't include any continue responses because they contain no content
-                .forEach(response -> drainResponse(response)); // drain all node responses that didn't get filtered out
+                .forEach(this::drainResponse); // drain all node responses that didn't get filtered out
     }
 
     private void drainResponse(final NodeResponse response) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ParameterContextEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ParameterContextEndpointMerger.java
new file mode 100644
index 0000000..4719965
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ParameterContextEndpointMerger.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.cluster.coordination.http.endpoints;
+
+import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
+import org.apache.nifi.cluster.manager.NodeResponse;
+import org.apache.nifi.cluster.protocol.NodeIdentifier;
+import org.apache.nifi.web.api.entity.ParameterContextEntity;
+
+import java.net.URI;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+public class ParameterContextEndpointMerger extends AbstractSingleEntityEndpoint<ParameterContextEntity> implements EndpointResponseMerger {
+    private static final Pattern PARAMETER_CONTEXT_URI = Pattern.compile("/nifi-api/parameter-contexts/[a-f0-9\\-]{36}");
+
+    @Override
+    public boolean canHandle(final URI uri, final String method) {
+        return ("GET".equalsIgnoreCase(method) || "DELETE".equalsIgnoreCase(method)) && PARAMETER_CONTEXT_URI.matcher(uri.getPath()).matches();
+    }
+
+    @Override
+    protected Class<ParameterContextEntity> getEntityClass() {
+        return ParameterContextEntity.class;
+    }
+
+    @Override
+    protected void mergeResponses(final ParameterContextEntity clientEntity, final Map<NodeIdentifier, ParameterContextEntity> entityMap, final Set<NodeResponse> successfulResponses,
+                                  final Set<NodeResponse> problematicResponses) {
+
+        ParameterContextMerger.merge(clientEntity, entityMap);
+
+    }
+
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ParameterContextMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ParameterContextMerger.java
new file mode 100644
index 0000000..7278b54
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ParameterContextMerger.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.cluster.coordination.http.endpoints;
+
+import org.apache.nifi.cluster.manager.PermissionsDtoMerger;
+import org.apache.nifi.cluster.manager.StatusMerger;
+import org.apache.nifi.cluster.protocol.NodeIdentifier;
+import org.apache.nifi.web.api.dto.AffectedComponentDTO;
+import org.apache.nifi.web.api.dto.ParameterContextDTO;
+import org.apache.nifi.web.api.dto.ParameterDTO;
+import org.apache.nifi.web.api.entity.AffectedComponentEntity;
+import org.apache.nifi.web.api.entity.ParameterContextEntity;
+import org.apache.nifi.web.api.entity.ParameterEntity;
+import org.apache.nifi.web.api.entity.ProcessGroupEntity;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class ParameterContextMerger {
+
+    public static void merge(final ParameterContextEntity target, final Map<NodeIdentifier, ParameterContextEntity> entityMap) {
+        final Map<NodeIdentifier, ParameterContextDTO> dtoMap = new HashMap<>();
+
+        for (final Map.Entry<NodeIdentifier, ParameterContextEntity> entry : entityMap.entrySet()) {
+            final NodeIdentifier nodeId = entry.getKey();
+            final ParameterContextEntity entity = entry.getValue();
+
+            PermissionsDtoMerger.mergePermissions(entity.getPermissions(), entity.getPermissions());
+
+            if (entity.getComponent() == null) {
+                target.setComponent(null);
+                continue;
+            }
+
+            dtoMap.put(nodeId, entity.getComponent());
+        }
+
+        if (target.getComponent() != null) {
+            merge(target.getComponent(), dtoMap);
+        }
+    }
+
+    public static void merge(final ParameterContextDTO target, final Map<NodeIdentifier, ParameterContextDTO> entityMap) {
+        final Map<String, ProcessGroupEntity> mergedBoundGroups = new HashMap<>();
+        final Map<String, Map<String, AffectedComponentEntity>> affectedComponentsByParameterName = new HashMap<>();
+
+        final Set<String> unwritableParameters = new HashSet<>();
+        for (final Map.Entry<NodeIdentifier, ParameterContextDTO> entry : entityMap.entrySet()) {
+            final NodeIdentifier nodeId = entry.getKey();
+            final ParameterContextDTO contextDto = entry.getValue();
+            if (contextDto == null) {
+                continue;
+            }
+
+            Set<ProcessGroupEntity> boundGroupEntities = contextDto.getBoundProcessGroups();
+            if (boundGroupEntities != null) {
+                for (final ProcessGroupEntity groupEntity : boundGroupEntities) {
+                    ProcessGroupEntity merged = mergedBoundGroups.get(groupEntity.getId());
+                    if (merged == null) {
+                        mergedBoundGroups.put(groupEntity.getId(), groupEntity);
+                        continue;
+                    }
+
+                    merge(merged, nodeId, groupEntity);
+                }
+            }
+
+            if (contextDto.getParameters() != null) {
+                for (final ParameterEntity parameterEntity : contextDto.getParameters()) {
+                    final ParameterDTO parameterDto = parameterEntity.getParameter();
+
+                    if (!parameterEntity.getCanWrite()) {
+                        unwritableParameters.add(parameterDto.getName());
+                    }
+
+                    final Map<String, AffectedComponentEntity> affectedComponentsById = affectedComponentsByParameterName.computeIfAbsent(parameterDto.getName(), key -> new HashMap<>());
+
+                    for (final AffectedComponentEntity referencingComponent : parameterDto.getReferencingComponents()) {
+                        AffectedComponentEntity mergedAffectedComponent = affectedComponentsById.get(referencingComponent.getId());
+                        if (mergedAffectedComponent == null) {
+                            affectedComponentsById.put(referencingComponent.getId(), referencingComponent);
+                            continue;
+                        }
+
+                        merge(mergedAffectedComponent, referencingComponent);
+                    }
+                }
+            }
+        }
+
+        target.setBoundProcessGroups(new HashSet<>(mergedBoundGroups.values()));
+
+        // Set the merged parameter dto's
+        for (final ParameterEntity parameterEntity : target.getParameters()) {
+            final ParameterDTO parameterDto = parameterEntity.getParameter();
+            final boolean unwritable = unwritableParameters.contains(parameterDto.getName());
+            if (unwritable) {
+                parameterEntity.setCanWrite(false);
+            }
+
+            final Map<String, AffectedComponentEntity> componentMap = affectedComponentsByParameterName.get(parameterDto.getName());
+            parameterDto.setReferencingComponents(new HashSet<>(componentMap.values()));
+        }
+    }
+
+
+    private static void merge(final ProcessGroupEntity merged, final NodeIdentifier additionalNodeId, final ProcessGroupEntity additional) {
+        PermissionsDtoMerger.mergePermissions(merged.getPermissions(), additional.getPermissions());
+
+        if (!Boolean.TRUE.equals(merged.getPermissions().getCanRead()) || additional.getComponent() == null) {
+            merged.setComponent(null);
+        }
+
+        StatusMerger.merge(merged.getStatus(), merged.getPermissions().getCanRead(), additional.getStatus(), additional.getPermissions().getCanRead(), additionalNodeId.getId(),
+            additionalNodeId.getApiAddress(), additionalNodeId.getApiPort());
+    }
+
+    static void merge(final AffectedComponentEntity merged, final AffectedComponentEntity additional) {
+        PermissionsDtoMerger.mergePermissions(merged.getPermissions(), additional.getPermissions());
+
+        if (!Boolean.TRUE.equals(merged.getPermissions().getCanRead()) || additional.getComponent() == null) {
+            merged.setComponent(null);
+            return;
+        }
+
+        final AffectedComponentDTO mergedComponent = merged.getComponent();
+        final AffectedComponentDTO additionalComponent = additional.getComponent();
+        mergedComponent.setActiveThreadCount(mergedComponent.getActiveThreadCount() + additionalComponent.getActiveThreadCount());
+
+        if (mergedComponent.getValidationErrors() == null) {
+            mergedComponent.setValidationErrors(new ArrayList<>());
+        }
+
+        if (additionalComponent.getValidationErrors() != null) {
+            mergedComponent.getValidationErrors().addAll(additionalComponent.getValidationErrors());
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ParameterContextUpdateEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ParameterContextUpdateEndpointMerger.java
new file mode 100644
index 0000000..3ea4aac
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ParameterContextUpdateEndpointMerger.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.cluster.coordination.http.endpoints;
+
+import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
+import org.apache.nifi.cluster.manager.NodeResponse;
+import org.apache.nifi.cluster.protocol.NodeIdentifier;
+import org.apache.nifi.web.api.dto.ParameterContextDTO;
+import org.apache.nifi.web.api.dto.ParameterContextUpdateRequestDTO;
+import org.apache.nifi.web.api.entity.AffectedComponentEntity;
+import org.apache.nifi.web.api.entity.ParameterContextUpdateRequestEntity;
+
+import java.net.URI;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+public class ParameterContextUpdateEndpointMerger extends AbstractSingleEntityEndpoint<ParameterContextUpdateRequestEntity> implements EndpointResponseMerger {
+    private static final Pattern PARAMETER_CONTEXT_URI = Pattern.compile("/nifi-api/parameter-contexts/[a-f0-9\\-]{36}");
+    private static final String PARAMETER_CONTEXTS_URI = "/nifi-api/parameter-contexts";
+
+    @Override
+    public boolean canHandle(final URI uri, final String method) {
+        return ("GET".equalsIgnoreCase(method) || "DELETE".equalsIgnoreCase(method)) && PARAMETER_CONTEXT_URI.matcher(uri.getPath()).matches()
+            || "POST".equalsIgnoreCase(method) && PARAMETER_CONTEXTS_URI.equals(method);
+    }
+
+    @Override
+    protected Class<ParameterContextUpdateRequestEntity> getEntityClass() {
+        return ParameterContextUpdateRequestEntity.class;
+    }
+
+    @Override
+    protected void mergeResponses(final ParameterContextUpdateRequestEntity clientEntity, final Map<NodeIdentifier, ParameterContextUpdateRequestEntity> entityMap,
+                                  final Set<NodeResponse> successfulResponses, final Set<NodeResponse> problematicResponses) {
+
+        final ParameterContextUpdateRequestDTO clientUpdateRequestDto = clientEntity.getRequest();
+
+        final Map<String, AffectedComponentEntity> affectedComponentEntities = new HashMap<>();
+        for (final ParameterContextUpdateRequestEntity requestEntity : entityMap.values()) {
+            final ParameterContextUpdateRequestDTO updateRequestDto = requestEntity.getRequest();
+            clientUpdateRequestDto.setComplete(clientUpdateRequestDto.isComplete() && updateRequestDto.isComplete());
+
+            if (updateRequestDto.getFailureReason() != null) {
+                clientUpdateRequestDto.setFailureReason(updateRequestDto.getFailureReason());
+            }
+
+            clientUpdateRequestDto.setLastUpdated(new Date(Math.min(clientUpdateRequestDto.getLastUpdated().getTime(), updateRequestDto.getLastUpdated().getTime())));
+            clientUpdateRequestDto.setPercentCompleted(Math.min(clientUpdateRequestDto.getPercentCompleted(), updateRequestDto.getPercentCompleted()));
+
+            // Merge the Affected Components.
+            for (final AffectedComponentEntity entity : requestEntity.getRequest().getAffectedComponents()) {
+                final AffectedComponentEntity mergedAffectedComponentEntity = affectedComponentEntities.get(entity.getId());
+                if (mergedAffectedComponentEntity == null) {
+                    affectedComponentEntities.put(entity.getId(), entity);
+                    return;
+                }
+
+                ParameterContextMerger.merge(mergedAffectedComponentEntity, entity);
+            }
+        }
+
+        final Map<NodeIdentifier, ParameterContextDTO> contextDtoMap = new HashMap<>();
+        entityMap.forEach( (nodeId, entity) -> contextDtoMap.put(nodeId, entity.getRequest().getParameterContext()));
+
+        ParameterContextMerger.merge(clientUpdateRequestDto.getParameterContext(), contextDtoMap);
+        clientUpdateRequestDto.setAffectedComponents(new HashSet<>(affectedComponentEntities.values()));
+    }
+
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ParameterContextValidationMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ParameterContextValidationMerger.java
new file mode 100644
index 0000000..d925d4b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ParameterContextValidationMerger.java
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.cluster.coordination.http.endpoints;
+
+import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
+import org.apache.nifi.cluster.manager.NodeResponse;
+import org.apache.nifi.cluster.manager.PermissionsDtoMerger;
+import org.apache.nifi.cluster.protocol.NodeIdentifier;
+import org.apache.nifi.web.api.dto.ComponentValidationResultDTO;
+import org.apache.nifi.web.api.dto.ParameterContextValidationRequestDTO;
+import org.apache.nifi.web.api.entity.ComponentValidationResultEntity;
+import org.apache.nifi.web.api.entity.ComponentValidationResultsEntity;
+import org.apache.nifi.web.api.entity.ParameterContextValidationRequestEntity;
+
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+public class ParameterContextValidationMerger extends AbstractSingleEntityEndpoint<ParameterContextValidationRequestEntity> implements EndpointResponseMerger {
+    public static final Pattern REQUESTS_URI_PATTERN = Pattern.compile("/nifi-api/parameter-contexts/validation-requests");
+    public static final Pattern REQUEST_BY_ID_URI_PATTERN = Pattern.compile("/nifi-api/parameter-contexts/validation-requests/[a-f0-9\\-]{36}");
+
+    @Override
+    public boolean canHandle(final URI uri, final String method) {
+        if ("POST".equalsIgnoreCase(method) && REQUESTS_URI_PATTERN.matcher(uri.getPath()).matches()) {
+            return true;
+        }
+
+        return (REQUEST_BY_ID_URI_PATTERN.matcher(uri.getPath()).matches());
+    }
+
+    @Override
+    protected Class<ParameterContextValidationRequestEntity> getEntityClass() {
+        return ParameterContextValidationRequestEntity.class;
+    }
+
+    @Override
+    protected void mergeResponses(final ParameterContextValidationRequestEntity clientEntity, final Map<NodeIdentifier, ParameterContextValidationRequestEntity> entityMap,
+                                  final Set<NodeResponse> successfulResponses, final Set<NodeResponse> problematicResponses) {
+
+        final ParameterContextValidationRequestDTO validationRequest = clientEntity.getRequest();
+
+        for (final ParameterContextValidationRequestEntity requestEntity : entityMap.values()) {
+            final ParameterContextValidationRequestDTO requestDto = requestEntity.getRequest();
+
+            if (!requestDto.isComplete()) {
+                validationRequest.setComplete(false);
+            }
+            if (requestDto.getFailureReason() != null) {
+                validationRequest.setFailureReason(requestDto.getFailureReason());
+            }
+            if (requestDto.getLastUpdated() != null && (validationRequest.getLastUpdated() == null || requestDto.getLastUpdated().after(validationRequest.getLastUpdated()))) {
+                validationRequest.setLastUpdated(requestDto.getLastUpdated());
+            }
+            if (requestDto.getPercentCompleted() < validationRequest.getPercentCompleted()) {
+                validationRequest.setPercentCompleted(requestDto.getPercentCompleted());
+                validationRequest.setState(requestDto.getState());
+            }
+        }
+
+        final ComponentValidationResultsEntity resultsEntity = validationRequest.getComponentValidationResults();
+        if (resultsEntity == null) {
+            return;
+        }
+
+        final List<ComponentValidationResultEntity> resultsEntities = resultsEntity.getValidationResults();
+        if (resultsEntities == null) {
+            return;
+        }
+
+        final Map<String, ComponentValidationResultEntity> resultsById = new HashMap<>();
+        for (final ComponentValidationResultEntity resultEntity : resultsEntities) {
+            resultsById.put(resultEntity.getId(), resultEntity);
+        }
+
+        for (final ParameterContextValidationRequestEntity requestEntity : entityMap.values()) {
+            final ComponentValidationResultsEntity validationResultsEntity = requestEntity.getRequest().getComponentValidationResults();
+            if (validationResultsEntity == null) {
+                continue;
+            }
+
+            for (final ComponentValidationResultEntity resultEntity : validationResultsEntity.getValidationResults()) {
+                final ComponentValidationResultEntity mergedResultEntity = resultsById.get(resultEntity.getId());
+                if (mergedResultEntity == null) {
+                    resultsById.put(resultEntity.getId(), resultEntity);
+                } else {
+                    merge(mergedResultEntity, resultEntity);
+                }
+            }
+        }
+
+        resultsEntity.setValidationResults(new ArrayList<>(resultsById.values()));
+    }
+
+    private void merge(final ComponentValidationResultEntity merged, final ComponentValidationResultEntity additional) {
+        if (merged.getComponent() == null) {
+            return;
+        }
+
+        PermissionsDtoMerger.mergePermissions(merged.getPermissions(), additional.getPermissions());
+
+        // If either entity doesn't have the component, then we cannot return the component because it was filtered out due to permissions.
+        if (additional.getComponent() == null) {
+            merged.setComponent(null);
+            return;
+        }
+
+        final ComponentValidationResultDTO mergedResultDto = merged.getComponent();
+        final ComponentValidationResultDTO additionalResultDto = additional.getComponent();
+
+        mergedResultDto.setActiveThreadCount(mergedResultDto.getActiveThreadCount() + additionalResultDto.getActiveThreadCount());
+
+        // Merge validation errors
+        Collection<String> mergedValidationErrors = mergedResultDto.getValidationErrors();
+        if (mergedValidationErrors == null) {
+            mergedValidationErrors = new ArrayList<>();
+            mergedResultDto.setValidationErrors(mergedValidationErrors);
+        }
+        if (additionalResultDto.getValidationErrors() != null) {
+            mergedValidationErrors.addAll(additionalResultDto.getValidationErrors());
+        }
+
+        // Merge resultant validation errors
+        Collection<String> mergedResultantValidationErrors = mergedResultDto.getResultantValidationErrors();
+        if (mergedResultantValidationErrors == null) {
+            mergedResultantValidationErrors = new ArrayList<>();
+            mergedResultDto.setResultantValidationErrors(mergedResultantValidationErrors);
+        }
+        if (additionalResultDto.getResultantValidationErrors() != null) {
+            mergedResultantValidationErrors.addAll(additionalResultDto.getResultantValidationErrors());
+        }
+
+        // Merge currently valid & results valid fields
+        mergedResultDto.setCurrentlyValid(mergedValidationErrors.isEmpty());
+        mergedResultDto.setResultsValid(mergedResultantValidationErrors.isEmpty());
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ParameterContextsEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ParameterContextsEndpointMerger.java
new file mode 100644
index 0000000..cf95580
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ParameterContextsEndpointMerger.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.cluster.coordination.http.endpoints;
+
+import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
+import org.apache.nifi.cluster.manager.NodeResponse;
+import org.apache.nifi.cluster.manager.PermissionsDtoMerger;
+import org.apache.nifi.cluster.protocol.NodeIdentifier;
+import org.apache.nifi.web.api.entity.ParameterContextEntity;
+import org.apache.nifi.web.api.entity.ParameterContextsEntity;
+
+import java.net.URI;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class ParameterContextsEndpointMerger extends AbstractSingleEntityEndpoint<ParameterContextsEntity> implements EndpointResponseMerger {
+    private static final String PARAMETER_CONTEXTS_URI = "/nifi-api/parameter-contexts";
+
+    @Override
+    public boolean canHandle(final URI uri, final String method) {
+        return "GET".equalsIgnoreCase(method) && PARAMETER_CONTEXTS_URI.equals(uri.getPath());
+    }
+
+    @Override
+    protected Class<ParameterContextsEntity> getEntityClass() {
+        return ParameterContextsEntity.class;
+    }
+
+    @Override
+    protected void mergeResponses(final ParameterContextsEntity clientEntity, final Map<NodeIdentifier, ParameterContextsEntity> entityMap, final Set<NodeResponse> successfulResponses,
+                                  final Set<NodeResponse> problematicResponses) {
+
+        final Map<String, ParameterContextEntity> contextEntities = new HashMap<>();
+        for (final ParameterContextsEntity contextsEntity : entityMap.values()) {
+            for (final ParameterContextEntity entity : contextsEntity.getParameterContexts()) {
+                final ParameterContextEntity mergedEntity = contextEntities.get(entity.getId());
+                if (mergedEntity == null) {
+                    contextEntities.put(entity.getId(), entity);
+                    continue;
+                }
+
+                merge(mergedEntity, entity);
+            }
+        }
+
+        clientEntity.setParameterContexts(new HashSet<>(contextEntities.values()));
+    }
+
+    private void merge(final ParameterContextEntity merged, final ParameterContextEntity additional) {
+        PermissionsDtoMerger.mergePermissions(merged.getPermissions(), additional.getPermissions());
+
+        if (merged.getComponent() == null || additional.getComponent() == null) {
+            merged.setComponent(null);
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java
index 66eec26..2ca8ef8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java
@@ -22,6 +22,7 @@ import com.fasterxml.jackson.core.JsonParser;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.commons.collections4.queue.CircularFifoQueue;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.cluster.coordination.ClusterCoordinator;
 import org.apache.nifi.cluster.coordination.ClusterTopologyEventListener;
 import org.apache.nifi.cluster.coordination.flow.FlowElection;
@@ -34,8 +35,8 @@ import org.apache.nifi.cluster.event.NodeEvent;
 import org.apache.nifi.cluster.exception.NoClusterCoordinatorException;
 import org.apache.nifi.cluster.firewall.ClusterNodeFirewall;
 import org.apache.nifi.cluster.manager.NodeResponse;
-import org.apache.nifi.cluster.manager.exception.IllegalNodeOffloadException;
 import org.apache.nifi.cluster.manager.exception.IllegalNodeDisconnectionException;
+import org.apache.nifi.cluster.manager.exception.IllegalNodeOffloadException;
 import org.apache.nifi.cluster.protocol.ComponentRevision;
 import org.apache.nifi.cluster.protocol.ConnectionRequest;
 import org.apache.nifi.cluster.protocol.ConnectionResponse;
@@ -50,10 +51,10 @@ import org.apache.nifi.cluster.protocol.message.ClusterWorkloadRequestMessage;
 import org.apache.nifi.cluster.protocol.message.ClusterWorkloadResponseMessage;
 import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage;
 import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage;
-import org.apache.nifi.cluster.protocol.message.OffloadMessage;
 import org.apache.nifi.cluster.protocol.message.DisconnectMessage;
 import org.apache.nifi.cluster.protocol.message.NodeConnectionStatusResponseMessage;
 import org.apache.nifi.cluster.protocol.message.NodeStatusChangeMessage;
+import org.apache.nifi.cluster.protocol.message.OffloadMessage;
 import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
 import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType;
 import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage;
@@ -126,7 +127,7 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
                                   final FlowElection flowElection, final ClusterNodeFirewall firewall, final RevisionManager revisionManager, final NiFiProperties nifiProperties,
                                   final ExtensionManager extensionManager, final NodeProtocolSender nodeProtocolSender) throws IOException {
         this(senderListener, eventReporter, leaderElectionManager, flowElection, firewall, revisionManager, nifiProperties, nodeProtocolSender,
-            StandardStateManagerProvider.create(nifiProperties, VariableRegistry.EMPTY_REGISTRY, extensionManager));
+            StandardStateManagerProvider.create(nifiProperties, VariableRegistry.EMPTY_REGISTRY, extensionManager, ParameterLookup.EMPTY));
     }
 
     public NodeClusterCoordinator(final ClusterCoordinationProtocolSenderListener senderListener, final EventReporter eventReporter, final LeaderElectionManager leaderElectionManager,
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/StatusMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/StatusMerger.java
index 0043ca0..946295a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/StatusMerger.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/StatusMerger.java
@@ -96,6 +96,10 @@ public class StatusMerger {
 
     public static void merge(final ProcessGroupStatusDTO target, final boolean targetReadablePermission, final ProcessGroupStatusDTO toMerge, final boolean toMergeReadablePermission,
                              final String nodeId, final String nodeAddress, final Integer nodeApiPort) {
+        if (toMerge == null) {
+            return;
+        }
+
         if (targetReadablePermission && !toMergeReadablePermission) {
             target.setId(toMerge.getId());
             target.setName(toMerge.getName());
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/flow/TestPopularVoteFlowElection.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/flow/TestPopularVoteFlowElection.java
index 962f521..fe675dc 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/flow/TestPopularVoteFlowElection.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/flow/TestPopularVoteFlowElection.java
@@ -17,21 +17,6 @@
 
 package org.apache.nifi.cluster.coordination.flow;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.ArgumentMatchers.anyString;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Paths;
-import java.util.HashSet;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
 import org.apache.nifi.cluster.protocol.DataFlow;
 import org.apache.nifi.cluster.protocol.NodeIdentifier;
 import org.apache.nifi.cluster.protocol.StandardDataFlow;
@@ -45,6 +30,22 @@ import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
 public class TestPopularVoteFlowElection {
 
     /**
@@ -183,35 +184,6 @@ public class TestPopularVoteFlowElection {
         }
     }
 
-    @Test
-    public void testDifferentPopulatedFlowsElection() throws IOException {
-        final ExtensionManager extensionManager = new StandardExtensionDiscoveringManager();
-        final FingerprintFactory fingerprintFactory = new FingerprintFactory(createEncryptorFromProperties(getNiFiProperties()), extensionManager);
-        final PopularVoteFlowElection election = new PopularVoteFlowElection(1, TimeUnit.MINUTES, 4, fingerprintFactory);
-        final byte[] nonEmptyCandidateA = Files.readAllBytes(Paths.get("src/test/resources/conf/controller-service-flow.xml"));
-        final byte[] nonEmptyCandidateB = Files.readAllBytes(Paths.get("src/test/resources/conf/reporting-task-flow.xml"));
-
-        for (int i = 0; i < 4; i++) {
-            assertFalse(election.isElectionComplete());
-            assertNull(election.getElectedDataFlow());
-
-            final DataFlow dataFlow;
-            if (i % 2 == 0) {
-                dataFlow = createDataFlow(nonEmptyCandidateA);
-            } else {
-                dataFlow = createDataFlow(nonEmptyCandidateB);
-            }
-
-            final DataFlow electedDataFlow = election.castVote(dataFlow, createNodeId(i));
-
-            if (i == 3) {
-                assertNotNull(electedDataFlow);
-                assertEquals(new String(nonEmptyCandidateA), new String(electedDataFlow.getFlow()));
-            } else {
-                assertNull(electedDataFlow);
-            }
-        }
-    }
 
     private NiFiProperties getNiFiProperties() {
         final NiFiProperties nifiProperties = mock(NiFiProperties.class);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml
index 39d040f..6371c7d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml
@@ -1,13 +1,13 @@
 <?xml version="1.0"?>
-<!-- Licensed to the Apache Software Foundation (ASF) under one or more contributor 
-license agreements. See the NOTICE file distributed with this work for additional 
-information regarding copyright ownership. The ASF licenses this file to 
-You under the Apache License, Version 2.0 (the "License"); you may not use 
-this file except in compliance with the License. You may obtain a copy of 
-the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required 
-by applicable law or agreed to in writing, software distributed under the 
-License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 
-OF ANY KIND, either express or implied. See the License for the specific 
+<!-- Licensed to the Apache Software Foundation (ASF) under one or more contributor
+license agreements. See the NOTICE file distributed with this work for additional
+information regarding copyright ownership. The ASF licenses this file to
+You under the Apache License, Version 2.0 (the "License"); you may not use
+this file except in compliance with the License. You may obtain a copy of
+the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required
+by applicable law or agreed to in writing, software distributed under the
+License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS
+OF ANY KIND, either express or implied. See the License for the specific
 language governing permissions and limitations under the License. -->
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
     <modelVersion>4.0.0</modelVersion>
@@ -29,6 +29,10 @@ language governing permissions and limitations under the License. -->
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-parameter</artifactId>
... 13297 lines suppressed ...