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

[01/15] nifi git commit: NIFI-4149 - Indicate if EL is evaluated against FFs or not - take into account input requirement for documentation rendering - Renamed variable registry scope and added comments - Doc + change in mock framework to check scope + u

Repository: nifi
Updated Branches:
  refs/heads/master 7ff38f690 -> 644133dc3


http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonRecordSetWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonRecordSetWriter.java
index 74f4352..c5cb82a 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonRecordSetWriter.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonRecordSetWriter.java
@@ -28,6 +28,7 @@ import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.schema.access.SchemaNotFoundException;
 import org.apache.nifi.serialization.DateTimeTextRecordSetWriter;
@@ -58,7 +59,7 @@ public class JsonRecordSetWriter extends DateTimeTextRecordSetWriter implements
     static final PropertyDescriptor PRETTY_PRINT_JSON = new PropertyDescriptor.Builder()
         .name("Pretty Print JSON")
         .description("Specifies whether or not the JSON should be pretty printed")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues("true", "false")
         .defaultValue("false")
         .required(true)

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextRecordSetWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextRecordSetWriter.java
index 1ed2aba..e5f8514 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextRecordSetWriter.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextRecordSetWriter.java
@@ -28,6 +28,7 @@ import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.schema.access.InheritSchemaFromRecord;
@@ -47,7 +48,7 @@ public class FreeFormTextRecordSetWriter extends SchemaRegistryRecordSetWriter i
         .name("Text")
         .description("The text to use when writing the results. This property will evaluate the Expression Language using any of the fields available in a Record.")
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .required(true)
         .build();
     static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder()
@@ -55,7 +56,7 @@ public class FreeFormTextRecordSetWriter extends SchemaRegistryRecordSetWriter i
         .description("The Character set to use when writing the data to the FlowFile")
         .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
         .defaultValue("UTF-8")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(true)
         .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-stateful-analysis-bundle/nifi-stateful-analysis-processors/src/main/java/org/apache/nifi/processors/stateful/analysis/AttributeRollingWindow.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-stateful-analysis-bundle/nifi-stateful-analysis-processors/src/main/java/org/apache/nifi/processors/stateful/analysis/AttributeRollingWindow.java b/nifi-nar-bundles/nifi-stateful-analysis-bundle/nifi-stateful-analysis-processors/src/main/java/org/apache/nifi/processors/stateful/analysis/AttributeRollingWindow.java
index d7f839a..3cc0330 100644
--- a/nifi-nar-bundles/nifi-stateful-analysis-bundle/nifi-stateful-analysis-processors/src/main/java/org/apache/nifi/processors/stateful/analysis/AttributeRollingWindow.java
+++ b/nifi-nar-bundles/nifi-stateful-analysis-bundle/nifi-stateful-analysis-processors/src/main/java/org/apache/nifi/processors/stateful/analysis/AttributeRollingWindow.java
@@ -28,6 +28,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.components.state.StateManager;
 import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -78,7 +79,7 @@ public class AttributeRollingWindow extends AbstractProcessor {
             .displayName("Value to track")
             .name("Value to track")
             .description("The expression on which to evaluate each FlowFile. The result of the expression will be added to the rolling window value.")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR)
             .required(true)
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-tcp-bundle/nifi-tcp-processors/src/main/java/org/apache/nifi/processors/gettcp/GetTCP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-tcp-bundle/nifi-tcp-processors/src/main/java/org/apache/nifi/processors/gettcp/GetTCP.java b/nifi-nar-bundles/nifi-tcp-bundle/nifi-tcp-processors/src/main/java/org/apache/nifi/processors/gettcp/GetTCP.java
index eecb662..2743de2 100644
--- a/nifi-nar-bundles/nifi-tcp-bundle/nifi-tcp-processors/src/main/java/org/apache/nifi/processors/gettcp/GetTCP.java
+++ b/nifi-nar-bundles/nifi-tcp-bundle/nifi-tcp-processors/src/main/java/org/apache/nifi/processors/gettcp/GetTCP.java
@@ -28,6 +28,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
+
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.SideEffectFree;
 import org.apache.nifi.annotation.behavior.TriggerSerially;
@@ -37,6 +38,7 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
 import org.apache.nifi.processor.DataUnit;
@@ -226,8 +228,12 @@ public class GetTCP extends AbstractSessionFactoryProcessor {
 
     @Override
     protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
-        return new PropertyDescriptor.Builder().required(false).name(propertyDescriptorName)
-                .addValidator(StandardValidators.NON_EMPTY_VALIDATOR).dynamic(true).expressionLanguageSupported(true)
+        return new PropertyDescriptor.Builder()
+                .required(false)
+                .name(propertyDescriptorName)
+                .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                .dynamic(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
                 .build();
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/src/main/java/org/apache/nifi/processors/attributes/UpdateAttribute.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/src/main/java/org/apache/nifi/processors/attributes/UpdateAttribute.java b/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/src/main/java/org/apache/nifi/processors/attributes/UpdateAttribute.java
index 1ea0b61..9ffd450 100644
--- a/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/src/main/java/org/apache/nifi/processors/attributes/UpdateAttribute.java
+++ b/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/src/main/java/org/apache/nifi/processors/attributes/UpdateAttribute.java
@@ -54,6 +54,7 @@ import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.components.state.StateManager;
 import org.apache.nifi.components.state.StateMap;
 import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.logging.ComponentLog;
@@ -79,7 +80,7 @@ import org.apache.nifi.update.attributes.serde.CriteriaSerDe;
 @InputRequirement(Requirement.INPUT_REQUIRED)
 @Tags({"attributes", "modification", "update", "delete", "Attribute Expression Language", "state"})
 @CapabilityDescription("Updates the Attributes for a FlowFile by using the Attribute Expression Language and/or deletes the attributes based on a regular expression")
-@DynamicProperty(name = "A FlowFile attribute to update", value = "The value to set it to", supportsExpressionLanguage = true,
+@DynamicProperty(name = "A FlowFile attribute to update", value = "The value to set it to", expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
         description = "Updates a FlowFile attribute specified by the Dynamic Property's key with the value specified by the Dynamic Property's value")
 @WritesAttribute(attribute = "See additional details", description = "This processor may write or remove zero or more attributes as described in additional details")
 @Stateful(scopes = {Scope.LOCAL}, description = "Gives the option to store values not only on the FlowFile but as stateful variables to be referenced in a recursive manner.")
@@ -151,7 +152,7 @@ public class UpdateAttribute extends AbstractProcessor implements Searchable {
             .description("Regular expression for attributes to be deleted from FlowFiles.  Existing attributes that match will be deleted regardless of whether they are updated by this processor.")
             .required(false)
             .addValidator(DELETE_PROPERTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final String STORE_STATE_NAME = "Store State";
@@ -205,7 +206,7 @@ public class UpdateAttribute extends AbstractProcessor implements Searchable {
                 .name(propertyDescriptorName)
                 .required(false)
                 .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
                 .dynamic(true);
 
         if (stateful) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-processors/src/main/java/org/apache/nifi/processors/websocket/PutWebSocket.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-processors/src/main/java/org/apache/nifi/processors/websocket/PutWebSocket.java b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-processors/src/main/java/org/apache/nifi/processors/websocket/PutWebSocket.java
index 1c91650..0cb698d 100644
--- a/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-processors/src/main/java/org/apache/nifi/processors/websocket/PutWebSocket.java
+++ b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-processors/src/main/java/org/apache/nifi/processors/websocket/PutWebSocket.java
@@ -48,6 +48,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -84,7 +85,7 @@ public class PutWebSocket extends AbstractProcessor {
                     "sent to all connected WebSocket peers for the WebSocket controller service endpoint.")
             .required(true)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("${" + ATTR_WS_SESSION_ID + "}")
             .build();
 
@@ -94,7 +95,7 @@ public class PutWebSocket extends AbstractProcessor {
             .description("A NiFi Expression to retrieve the id of a WebSocket ControllerService.")
             .required(true)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("${" + ATTR_WS_CS_ID + "}")
             .build();
 
@@ -104,7 +105,7 @@ public class PutWebSocket extends AbstractProcessor {
             .description("A NiFi Expression to retrieve the endpoint id of a WebSocket ControllerService.")
             .required(true)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("${" + ATTR_WS_ENDPOINT_ID + "}")
             .build();
 
@@ -115,7 +116,7 @@ public class PutWebSocket extends AbstractProcessor {
             .required(true)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
             .defaultValue(WebSocketMessage.Type.TEXT.toString())
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final Relationship REL_SUCCESS = new Relationship.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/main/java/org/apache/nifi/websocket/jetty/JettyWebSocketClient.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/main/java/org/apache/nifi/websocket/jetty/JettyWebSocketClient.java b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/main/java/org/apache/nifi/websocket/jetty/JettyWebSocketClient.java
index 281b016..2a6bb84 100644
--- a/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/main/java/org/apache/nifi/websocket/jetty/JettyWebSocketClient.java
+++ b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/main/java/org/apache/nifi/websocket/jetty/JettyWebSocketClient.java
@@ -24,6 +24,7 @@ import org.apache.nifi.annotation.lifecycle.OnShutdown;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.ssl.SSLContextService;
@@ -59,7 +60,7 @@ public class JettyWebSocketClient extends AbstractJettyWebSocketService implemen
             .displayName("WebSocket URI")
             .description("The WebSocket URI this client connects to.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.URI_VALIDATOR)
             .addValidator((subject, input, context) -> {
                 final ValidationResult.Builder result = new ValidationResult.Builder()
@@ -82,7 +83,7 @@ public class JettyWebSocketClient extends AbstractJettyWebSocketService implemen
             .displayName("Connection Timeout")
             .description("The timeout to connect the WebSocket URI.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
             .defaultValue("3 sec")
             .build();
@@ -97,7 +98,7 @@ public class JettyWebSocketClient extends AbstractJettyWebSocketService implemen
                     " so that a WebSocket client can reuse the same session id transparently after it reconnects successfully. " +
                     " The maintenance activity is executed until corresponding processors or this controller service is stopped.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
             .defaultValue("10 sec")
             .build();
@@ -142,10 +143,10 @@ public class JettyWebSocketClient extends AbstractJettyWebSocketService implemen
         client.start();
         activeSessions.clear();
 
-        webSocketUri = new URI(context.getProperty(WS_URI).getValue());
-        connectionTimeoutMillis = context.getProperty(CONNECTION_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS);
+        webSocketUri = new URI(context.getProperty(WS_URI).evaluateAttributeExpressions().getValue());
+        connectionTimeoutMillis = context.getProperty(CONNECTION_TIMEOUT).evaluateAttributeExpressions().asTimePeriod(TimeUnit.MILLISECONDS);
 
-        final Long sessionMaintenanceInterval = context.getProperty(SESSION_MAINTENANCE_INTERVAL).asTimePeriod(TimeUnit.MILLISECONDS);
+        final Long sessionMaintenanceInterval = context.getProperty(SESSION_MAINTENANCE_INTERVAL).evaluateAttributeExpressions().asTimePeriod(TimeUnit.MILLISECONDS);
 
         sessionMaintenanceScheduler = Executors.newSingleThreadScheduledExecutor();
         sessionMaintenanceScheduler.scheduleAtFixedRate(() -> {

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/main/java/org/apache/nifi/websocket/jetty/JettyWebSocketServer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/main/java/org/apache/nifi/websocket/jetty/JettyWebSocketServer.java b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/main/java/org/apache/nifi/websocket/jetty/JettyWebSocketServer.java
index 068be02..5a2dbbf 100644
--- a/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/main/java/org/apache/nifi/websocket/jetty/JettyWebSocketServer.java
+++ b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/main/java/org/apache/nifi/websocket/jetty/JettyWebSocketServer.java
@@ -30,6 +30,7 @@ import org.apache.nifi.annotation.lifecycle.OnShutdown;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.ssl.SSLContextService;
 import org.apache.nifi.websocket.WebSocketConfigurationException;
@@ -90,7 +91,7 @@ public class JettyWebSocketServer extends AbstractJettyWebSocketService implemen
             .displayName("Listen Port")
             .description("The port number on which this WebSocketServer listens to.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.PORT_VALIDATOR)
             .build();
 
@@ -180,7 +181,7 @@ public class JettyWebSocketServer extends AbstractJettyWebSocketService implemen
 
         server.setHandler(handlerCollection);
 
-        listenPort = context.getProperty(LISTEN_PORT).asInteger();
+        listenPort = context.getProperty(LISTEN_PORT).evaluateAttributeExpressions().asInteger();
         final SslContextFactory sslContextFactory = createSslFactory(context);
 
         final ServerConnector serverConnector = createConnector(sslContextFactory, listenPort);


[02/15] nifi git commit: NIFI-4149 - Indicate if EL is evaluated against FFs or not - take into account input requirement for documentation rendering - Renamed variable registry scope and added comments - Doc + change in mock framework to check scope + u

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestReplaceText.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestReplaceText.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestReplaceText.java
index f36e306..3755883 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestReplaceText.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestReplaceText.java
@@ -16,14 +16,6 @@
  */
 package org.apache.nifi.processors.standard;
 
-import org.apache.nifi.util.MockFlowFile;
-import org.apache.nifi.util.TestRunner;
-import org.apache.nifi.util.TestRunners;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-
 import java.io.File;
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
@@ -34,16 +26,35 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.regex.Pattern;
 
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
 public class TestReplaceText {
 
     @Rule
     public ExpectedException exception = ExpectedException.none();
 
-    @Test
-    public void testConfigurationCornerCase() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+    public TestRunner getRunner() {
+        TestRunner runner = TestRunners.newTestRunner(ReplaceText.class);
+
+        /**
+         * we have to disable validation of expression language because the scope of the evaluation
+         * for the search value depends of another property (the evaluation mode). If not disabling
+         * the validation, it'll throw an error about the eval
+         */
         runner.setValidateExpressionUsage(false);
 
+        return runner;
+    }
+
+    @Test
+    public void testConfigurationCornerCase() throws IOException {
+        final TestRunner runner = getRunner();
         runner.run();
         runner.enqueue(Paths.get("src/test/resources/hello.txt"));
         runner.run();
@@ -55,8 +66,7 @@ public class TestReplaceText {
 
     @Test
     public void testSimple() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, "ell");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "lle");
 
@@ -70,8 +80,7 @@ public class TestReplaceText {
 
     @Test
     public void testWithEscaped$InReplacement() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, "(?s:^.*$)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "a\\$b");
 
@@ -85,8 +94,7 @@ public class TestReplaceText {
 
     @Test
     public void testWithUnEscaped$InReplacement() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, "(?s:^.*$)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "a$b");
 
@@ -100,8 +108,7 @@ public class TestReplaceText {
 
     @Test
     public void testPrependSimple() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "TEST");
         runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.PREPEND);
 
@@ -115,8 +122,7 @@ public class TestReplaceText {
 
     @Test
     public void testPrependLineByLine() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "_");
         runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.PREPEND);
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
@@ -131,8 +137,7 @@ public class TestReplaceText {
 
     @Test
     public void testAppendSimple() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "TEST");
         runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.APPEND);
 
@@ -146,8 +151,7 @@ public class TestReplaceText {
 
     @Test
     public void testAppendWithCarriageReturn() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "!");
         runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.APPEND);
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
@@ -162,8 +166,7 @@ public class TestReplaceText {
 
     @Test
     public void testAppendWithNewLine() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "!");
         runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.APPEND);
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
@@ -178,8 +181,7 @@ public class TestReplaceText {
 
     @Test
     public void testAppendWithCarriageReturnNewLine() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "!");
         runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.APPEND);
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
@@ -194,8 +196,7 @@ public class TestReplaceText {
 
     @Test
     public void testLiteralSimple() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, "ell");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "lle");
         runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.LITERAL_REPLACE);
@@ -210,8 +211,7 @@ public class TestReplaceText {
 
     @Test
     public void testLiteralBackReference() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, "ell");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "[$1]");
         runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.LITERAL_REPLACE);
@@ -226,8 +226,7 @@ public class TestReplaceText {
 
     @Test
     public void testLiteral() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, ".ell.");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "test");
         runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.LITERAL_REPLACE);
@@ -246,8 +245,7 @@ public class TestReplaceText {
 
     @Test
     public void testBackReference() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, "(ell)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "[$1]");
 
@@ -261,8 +259,7 @@ public class TestReplaceText {
 
     @Test
     public void testBackRefFollowedByNumbers() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         String expected = "Hell23o, World!";
         runner.setProperty(ReplaceText.SEARCH_VALUE, "(ell)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "$123");
@@ -280,8 +277,7 @@ public class TestReplaceText {
 
     @Test
     public void testBackRefWithNoCapturingGroup() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, "ell");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "$0123");
 
@@ -297,8 +293,7 @@ public class TestReplaceText {
 
     @Test
     public void testReplacementWithExpressionLanguage() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, "${replaceKey}");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "GoodBye");
 
@@ -315,8 +310,7 @@ public class TestReplaceText {
 
     @Test
     public void testReplacementWithExpressionLanguageIsEscaped() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, "(ell)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "[${abc}]");
 
@@ -333,8 +327,7 @@ public class TestReplaceText {
 
     @Test
     public void testRegexWithExpressionLanguage() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, "${replaceKey}");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${replaceValue}");
 
@@ -352,8 +345,7 @@ public class TestReplaceText {
 
     @Test
     public void testRegexWithExpressionLanguageIsEscaped() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, "${replaceKey}");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${replaceValue}");
 
@@ -371,8 +363,7 @@ public class TestReplaceText {
 
     @Test
     public void testBackReferenceWithTooLargeOfIndexIsEscaped() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, "(ell)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "$1$2");
 
@@ -390,8 +381,7 @@ public class TestReplaceText {
 
     @Test
     public void testBackReferenceWithInvalidReferenceIsEscaped() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, "(ell)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "$d");
 
@@ -409,8 +399,7 @@ public class TestReplaceText {
 
     @Test
     public void testEscapingDollarSign() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, "(ell)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "\\$1");
 
@@ -428,8 +417,7 @@ public class TestReplaceText {
 
     @Test
     public void testReplaceWithEmptyString() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, "(ell)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "");
 
@@ -443,8 +431,7 @@ public class TestReplaceText {
 
     @Test
     public void testWithNoMatch() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, "Z");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "Morning");
 
@@ -458,8 +445,7 @@ public class TestReplaceText {
 
     @Test
     public void testWithMultipleMatches() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, "l");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "R");
 
@@ -473,8 +459,7 @@ public class TestReplaceText {
 
     @Test
     public void testAttributeToContent() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, ".*");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${abc}");
 
@@ -491,8 +476,7 @@ public class TestReplaceText {
 
     @Test
     public void testRoutesToFailureIfTooLarge() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, "[123]");
         runner.setProperty(ReplaceText.MAX_BUFFER_SIZE, "1 b");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${abc}");
@@ -508,8 +492,7 @@ public class TestReplaceText {
 
     @Test
     public void testRoutesToSuccessIfTooLargeButRegexIsDotAsterisk() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, ".*");
         runner.setProperty(ReplaceText.MAX_BUFFER_SIZE, "1 b");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${abc}");
@@ -527,8 +510,7 @@ public class TestReplaceText {
 
     @Test
     public void testProblematicCase1() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, ".*");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${filename}\t${now():format(\"yyyy/MM/dd'T'HHmmss'Z'\")}\t${fileSize}\n");
 
@@ -548,8 +530,7 @@ public class TestReplaceText {
 
     @Test
     public void testGetExistingContent() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, "(?s)(^.*)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "attribute header\n\n${filename}\n\ndata header\n\n$1\n\nfooter");
 
@@ -568,8 +549,7 @@ public class TestReplaceText {
 
     @Test
     public void testReplaceWithinCurlyBraces() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, ".+");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "{ ${filename} }");
 
@@ -589,8 +569,7 @@ public class TestReplaceText {
         final String defaultValue = "default-replacement-value";
 
         // leave the default regex settings
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, defaultValue);
 
         final Map<String, String> attributes = new HashMap<>();
@@ -608,8 +587,7 @@ public class TestReplaceText {
         final String defaultValue = "default-replacement-value";
 
         // leave the default regex settings
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, defaultValue);
 
         final Map<String, String> attributes = new HashMap<>();
@@ -626,8 +604,7 @@ public class TestReplaceText {
 
     @Test
     public void testSimpleLineByLine() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.SEARCH_VALUE, "odo");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "ood");
@@ -642,8 +619,7 @@ public class TestReplaceText {
 
     @Test
     public void testPrependSimpleLineByLine() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.PREPEND);
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "TEST ");
@@ -658,8 +634,7 @@ public class TestReplaceText {
 
     @Test
     public void testAppendSimpleLineByLine() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.APPEND);
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, " TEST");
@@ -674,8 +649,7 @@ public class TestReplaceText {
 
     @Test
     public void testAppendEndlineCR() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "TEST");
         runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.APPEND);
@@ -690,8 +664,7 @@ public class TestReplaceText {
 
     @Test
     public void testAppendEndlineCRLF() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "TEST");
         runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.APPEND);
@@ -706,8 +679,7 @@ public class TestReplaceText {
 
     @Test
     public void testSimpleLiteral() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.SEARCH_VALUE, "odo");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "ood");
@@ -723,8 +695,7 @@ public class TestReplaceText {
 
     @Test
     public void testLiteralBackReferenceLineByLine() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.SEARCH_VALUE, "jo");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "[$1]");
@@ -741,8 +712,7 @@ public class TestReplaceText {
 
     @Test
     public void testLiteralLineByLine() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.SEARCH_VALUE, ".ell.");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "test");
@@ -758,8 +728,7 @@ public class TestReplaceText {
 
     @Test
     public void testBackReferenceLineByLine() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.SEARCH_VALUE, "(DODO)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "[$1]");
@@ -774,8 +743,7 @@ public class TestReplaceText {
 
     @Test
     public void testReplacementWithExpressionLanguageIsEscapedLineByLine() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.SEARCH_VALUE, "(jo)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "[${abc}]");
@@ -793,8 +761,7 @@ public class TestReplaceText {
 
     @Test
     public void testRegexWithExpressionLanguageLineByLine() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.SEARCH_VALUE, "${replaceKey}");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${replaceValue}");
@@ -813,8 +780,7 @@ public class TestReplaceText {
 
     @Test
     public void testRegexWithExpressionLanguageIsEscapedLineByLine() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.SEARCH_VALUE, "${replaceKey}");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${replaceValue}");
@@ -833,8 +799,7 @@ public class TestReplaceText {
 
     @Test
     public void testBackReferenceWithTooLargeOfIndexIsEscapedLineByLine() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.SEARCH_VALUE, "(lu)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "$1$2");
@@ -853,8 +818,7 @@ public class TestReplaceText {
 
     @Test
     public void testBackReferenceWithInvalidReferenceIsEscapedLineByLine() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.SEARCH_VALUE, "(ew)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "$d");
@@ -873,8 +837,7 @@ public class TestReplaceText {
 
     @Test
     public void testEscapingDollarSignLineByLine() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.SEARCH_VALUE, "(DO)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "\\$1");
@@ -893,8 +856,7 @@ public class TestReplaceText {
 
     @Test
     public void testReplaceWithEmptyStringLineByLine() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.SEARCH_VALUE, "(jo)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "");
@@ -909,8 +871,7 @@ public class TestReplaceText {
 
     @Test
     public void testWithNoMatchLineByLine() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.SEARCH_VALUE, "Z");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "Morning");
@@ -925,8 +886,7 @@ public class TestReplaceText {
 
     @Test
     public void testWithMultipleMatchesLineByLine() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.SEARCH_VALUE, "l");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "R");
@@ -941,8 +901,7 @@ public class TestReplaceText {
 
     @Test
     public void testAttributeToContentLineByLine() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.SEARCH_VALUE, ".*");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${abc}");
@@ -960,8 +919,7 @@ public class TestReplaceText {
 
     @Test
     public void testAttributeToContentWindows() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.SEARCH_VALUE, ".*");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${abc}");
@@ -979,8 +937,7 @@ public class TestReplaceText {
 
     @Test
     public void testProblematicCase1LineByLine() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.SEARCH_VALUE, ".*");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${filename}\t${now():format(\"yyyy/MM/dd'T'HHmmss'Z'\")}\t${fileSize}\n");
@@ -1001,8 +958,7 @@ public class TestReplaceText {
 
     @Test
     public void testGetExistingContentLineByLine() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.SEARCH_VALUE, "(?s)(^.*)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "attribute header\n\n${filename}\n\ndata header\n\n$1\n\nfooter\n");
@@ -1023,8 +979,7 @@ public class TestReplaceText {
 
     @Test
     public void testCapturingGroupInExpressionLanguage() {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.SEARCH_VALUE, "(.*?),(.*?),(\\d+.*)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "$1,$2,${ '$3':toDate('ddMMMyyyy'):format('yyyy/MM/dd') }");
@@ -1049,8 +1004,7 @@ public class TestReplaceText {
 
     @Test
     public void testCapturingGroupInExpressionLanguage2() {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.SEARCH_VALUE, "(.*)/(.*?).jpg");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "$1/${ '$2':substring(0,1) }.png");
@@ -1073,8 +1027,7 @@ public class TestReplaceText {
 
     @Test
     public void testAlwaysReplaceEntireText() {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.ENTIRE_TEXT);
         runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.ALWAYS_REPLACE);
         runner.setProperty(ReplaceText.SEARCH_VALUE, "i do not exist anywhere in the text");
@@ -1093,8 +1046,7 @@ public class TestReplaceText {
 
     @Test
     public void testAlwaysReplaceLineByLine() {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
         runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.ALWAYS_REPLACE);
         runner.setProperty(ReplaceText.SEARCH_VALUE, "i do not exist anywhere in the text");
@@ -1116,8 +1068,7 @@ public class TestReplaceText {
         // Test the old Default Regex and with a custom Replacement Value that should fail because the
         // Perl regex "(?s:^.*$)" must be written "(?s)(^.*$)" in Java for there to be a capture group.
         //      private static final String DEFAULT_REGEX = "(?s:^.*$)";
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, "(?s:^.*$)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${'$1':toUpper()}"); // should uppercase group but there is none
         runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.REGEX_REPLACE);
@@ -1134,8 +1085,7 @@ public class TestReplaceText {
     @Test
     public void testRegexWithGoodCaptureGroup() throws IOException {
         // Test the new Default Regex and with a custom Replacement Values that should succeed.
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, "(?s)(^.*$)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${'$1':toUpper()}"); // will uppercase group with good Java regex
         runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.REGEX_REPLACE);
@@ -1153,8 +1103,7 @@ public class TestReplaceText {
     public void testRegexNoCaptureDefaultReplacement() throws IOException {
         // Test the old Default Regex and new Default Regex with the default replacement.  This should fail
         // because the regex does not create a capture group.
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, "(?s:^.*$)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "$1");
         runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.REGEX_REPLACE);
@@ -1168,7 +1117,7 @@ public class TestReplaceText {
 
     @Test
     public void testProcessorConfigurationRegexNotValid() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceText.SEARCH_VALUE, "(?<!\\),*");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "hello");
         runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.REGEX_REPLACE);
@@ -1196,8 +1145,7 @@ public class TestReplaceText {
 
     @Test
     public void testBackReferenceEscapeWithRegexReplaceUsingEL() throws Exception {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
+        final TestRunner runner = getRunner();
 
         runner.setProperty(ReplaceText.SEARCH_VALUE, "(?s)(^.*$)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${'$1':toUpper()}");

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestReplaceTextWithMapping.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestReplaceTextWithMapping.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestReplaceTextWithMapping.java
index 7a480a8..fad370a 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestReplaceTextWithMapping.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestReplaceTextWithMapping.java
@@ -28,14 +28,26 @@ import java.util.Map;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
-
 import org.junit.Test;
 
 public class TestReplaceTextWithMapping {
 
+    public TestRunner getRunner() {
+        TestRunner runner = TestRunners.newTestRunner(ReplaceTextWithMapping.class);
+
+        /**
+         * we have to disable validation of expression language because the processor will
+         * need to evaluate the REGEX field with AND without flow files. If not disabled,
+         * the test will throw an error about the evaluation scope
+         */
+        runner.setValidateExpressionUsage(false);
+
+        return runner;
+    }
+
     @Test
     public void testSimple() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceTextWithMapping());
+        final TestRunner runner = getRunner();
         final String mappingFile = Paths.get("src/test/resources/TestReplaceTextWithMapping/color-fruit-mapping.txt").toFile().getAbsolutePath();
         runner.setProperty(ReplaceTextWithMapping.MAPPING_FILE, mappingFile);
 
@@ -54,7 +66,7 @@ public class TestReplaceTextWithMapping {
 
     @Test
     public void testExpressionLanguageInText() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceTextWithMapping());
+        final TestRunner runner = getRunner();
         final String mappingFile = Paths.get("src/test/resources/TestReplaceTextWithMapping/color-fruit-mapping.txt").toFile().getAbsolutePath();
         runner.setProperty(ReplaceTextWithMapping.MAPPING_FILE, mappingFile);
 
@@ -72,7 +84,7 @@ public class TestReplaceTextWithMapping {
 
     @Test
     public void testExpressionLanguageInText2() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceTextWithMapping());
+        final TestRunner runner = getRunner();
         final String mappingFile = Paths.get("src/test/resources/TestReplaceTextWithMapping/color-fruit-mapping.txt").toFile().getAbsolutePath();
         runner.setProperty(ReplaceTextWithMapping.MAPPING_FILE, mappingFile);
         runner.setProperty(ReplaceTextWithMapping.REGEX, "\\|(.*?)\\|");
@@ -92,7 +104,7 @@ public class TestReplaceTextWithMapping {
 
     @Test
     public void testExpressionLanguageInText3() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceTextWithMapping());
+        final TestRunner runner = getRunner();
         final String mappingFile = Paths.get("src/test/resources/TestReplaceTextWithMapping/color-fruit-mapping.txt").toFile().getAbsolutePath();
         runner.setProperty(ReplaceTextWithMapping.MAPPING_FILE, mappingFile);
         runner.setProperty(ReplaceTextWithMapping.REGEX, ".*\\|(.*?)\\|.*");
@@ -112,7 +124,7 @@ public class TestReplaceTextWithMapping {
 
     @Test
     public void testWithMatchingGroupAndContext() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceTextWithMapping());
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceTextWithMapping.REGEX, "-(.*?)-");
         runner.setProperty(ReplaceTextWithMapping.MATCHING_GROUP_FOR_LOOKUP_KEY, "1");
         runner.setProperty(ReplaceTextWithMapping.MAPPING_FILE, Paths.get("src/test/resources/TestReplaceTextWithMapping/color-fruit-mapping.txt").toFile().getAbsolutePath());
@@ -132,7 +144,7 @@ public class TestReplaceTextWithMapping {
 
     @Test
     public void testBackReference() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceTextWithMapping());
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceTextWithMapping.REGEX, "(\\S+)");
         runner.setProperty(ReplaceTextWithMapping.MATCHING_GROUP_FOR_LOOKUP_KEY, "1");
         runner.setProperty(ReplaceTextWithMapping.MAPPING_FILE, Paths.get("src/test/resources/TestReplaceTextWithMapping/color-fruit-backreference-mapping.txt").toFile().getAbsolutePath());
@@ -152,7 +164,7 @@ public class TestReplaceTextWithMapping {
 
     @Test
     public void testRoutesToFailureIfTooLarge() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceTextWithMapping());
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceTextWithMapping.REGEX, "[123]");
         runner.setProperty(ReplaceTextWithMapping.MAX_BUFFER_SIZE, "1 b");
         runner.setProperty(ReplaceTextWithMapping.MAPPING_FILE, Paths.get("src/test/resources/TestReplaceTextWithMapping/color-fruit-mapping.txt").toFile().getAbsolutePath());
@@ -168,7 +180,7 @@ public class TestReplaceTextWithMapping {
 
     @Test
     public void testBackReferenceWithTooLargeOfIndexIsEscaped() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceTextWithMapping());
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceTextWithMapping.REGEX, "-(.*?)-");
         runner.setProperty(ReplaceTextWithMapping.MATCHING_GROUP_FOR_LOOKUP_KEY, "1");
         runner.setProperty(ReplaceTextWithMapping.MAPPING_FILE, Paths.get("src/test/resources/TestReplaceTextWithMapping/color-fruit-excessive-backreference-mapping.txt").toFile().getAbsolutePath());
@@ -188,7 +200,7 @@ public class TestReplaceTextWithMapping {
 
     @Test
     public void testBackReferenceWithTooLargeOfIndexIsEscapedSimple() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceTextWithMapping());
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceTextWithMapping.MAPPING_FILE,
                 Paths.get("src/test/resources/TestReplaceTextWithMapping/color-fruit-excessive-backreference-mapping-simple.txt").toFile().getAbsolutePath());
 
@@ -207,7 +219,7 @@ public class TestReplaceTextWithMapping {
 
     @Test
     public void testBackReferenceWithInvalidReferenceIsEscaped() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceTextWithMapping());
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceTextWithMapping.REGEX, "(\\S+)");
         runner.setProperty(ReplaceTextWithMapping.MATCHING_GROUP_FOR_LOOKUP_KEY, "1");
         runner.setProperty(ReplaceTextWithMapping.MAPPING_FILE, Paths.get("src/test/resources/TestReplaceTextWithMapping/color-fruit-invalid-backreference-mapping.txt").toFile().getAbsolutePath());
@@ -227,7 +239,7 @@ public class TestReplaceTextWithMapping {
 
     @Test
     public void testEscapingDollarSign() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceTextWithMapping());
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceTextWithMapping.REGEX, "-(.*?)-");
         runner.setProperty(ReplaceTextWithMapping.MATCHING_GROUP_FOR_LOOKUP_KEY, "1");
         runner.setProperty(ReplaceTextWithMapping.MAPPING_FILE, Paths.get("src/test/resources/TestReplaceTextWithMapping/color-fruit-escaped-dollar-mapping.txt").toFile().getAbsolutePath());
@@ -247,7 +259,7 @@ public class TestReplaceTextWithMapping {
 
     @Test
     public void testEscapingDollarSignSimple() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceTextWithMapping());
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceTextWithMapping.MAPPING_FILE, Paths.get("src/test/resources/TestReplaceTextWithMapping/color-fruit-escaped-dollar-mapping.txt").toFile().getAbsolutePath());
 
         runner.enqueue(Paths.get("src/test/resources/TestReplaceTextWithMapping/colors-without-dashes.txt"));
@@ -265,7 +277,7 @@ public class TestReplaceTextWithMapping {
 
     @Test
     public void testReplaceWithEmptyString() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceTextWithMapping());
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceTextWithMapping.MAPPING_FILE, Paths.get("src/test/resources/TestReplaceTextWithMapping/color-fruit-blank-mapping.txt").toFile().getAbsolutePath());
 
         runner.enqueue(Paths.get("src/test/resources/TestReplaceTextWithMapping/colors-without-dashes.txt"));
@@ -283,7 +295,7 @@ public class TestReplaceTextWithMapping {
 
     @Test
     public void testReplaceWithSpaceInString() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceTextWithMapping());
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceTextWithMapping.MAPPING_FILE, Paths.get("src/test/resources/TestReplaceTextWithMapping/color-fruit-space-mapping.txt").toFile().getAbsolutePath());
 
         runner.enqueue(Paths.get("src/test/resources/TestReplaceTextWithMapping/colors-without-dashes.txt"));
@@ -301,7 +313,7 @@ public class TestReplaceTextWithMapping {
 
     @Test
     public void testWithNoMatch() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceTextWithMapping());
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceTextWithMapping.REGEX, "-(.*?)-");
         runner.setProperty(ReplaceTextWithMapping.MATCHING_GROUP_FOR_LOOKUP_KEY, "1");
         runner.setProperty(ReplaceTextWithMapping.MAPPING_FILE, Paths.get("src/test/resources/TestReplaceTextWithMapping/color-fruit-no-match-mapping.txt").toFile().getAbsolutePath());
@@ -319,7 +331,7 @@ public class TestReplaceTextWithMapping {
 
     @Test(expected = java.lang.AssertionError.class)
     public void testMatchingGroupForLookupKeyTooLarge() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceTextWithMapping());
+        final TestRunner runner = getRunner();
         runner.setProperty(ReplaceTextWithMapping.REGEX, "-(.*?)-");
         runner.setProperty(ReplaceTextWithMapping.MATCHING_GROUP_FOR_LOOKUP_KEY, "2");
         runner.setProperty(ReplaceTextWithMapping.MAPPING_FILE, Paths.get("src/test/resources/TestReplaceTextWithMapping/color-mapping.txt").toFile().getAbsolutePath());

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestUpdateRecord.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestUpdateRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestUpdateRecord.java
index ab16b96..e732695 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestUpdateRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestUpdateRecord.java
@@ -68,7 +68,6 @@ public class TestUpdateRecord {
     public void testLiteralReplacementValue() {
         runner.setProperty("/name", "Jane Doe");
         runner.enqueue("");
-        runner.setValidateExpressionUsage(false);
 
         readerService.addRecord("John Doe", 35);
         runner.run();
@@ -193,7 +192,6 @@ public class TestUpdateRecord {
     public void testUpdateInArray() throws InitializationException, IOException {
         final JsonTreeReader jsonReader = new JsonTreeReader();
         runner.addControllerService("reader", jsonReader);
-        runner.setValidateExpressionUsage(false);
 
         final String inputSchemaText = new String(Files.readAllBytes(Paths.get("src/test/resources/TestUpdateRecord/schema/person-with-address.avsc")));
         final String outputSchemaText = new String(Files.readAllBytes(Paths.get("src/test/resources/TestUpdateRecord/schema/person-with-address.avsc")));
@@ -224,7 +222,6 @@ public class TestUpdateRecord {
     public void testUpdateInNullArray() throws InitializationException, IOException {
         final JsonTreeReader jsonReader = new JsonTreeReader();
         runner.addControllerService("reader", jsonReader);
-        runner.setValidateExpressionUsage(false);
 
         final String inputSchemaText = new String(Files.readAllBytes(Paths.get("src/test/resources/TestUpdateRecord/schema/person-with-address.avsc")));
         final String outputSchemaText = new String(Files.readAllBytes(Paths.get("src/test/resources/TestUpdateRecord/schema/person-with-address.avsc")));
@@ -467,7 +464,6 @@ public class TestUpdateRecord {
     public void testUpdateSimpleArray() throws InitializationException, IOException {
         final JsonTreeReader jsonReader = new JsonTreeReader();
         runner.addControllerService("reader", jsonReader);
-        runner.setValidateExpressionUsage(false);
 
         final String inputSchemaText = new String(Files.readAllBytes(Paths.get("src/test/resources/TestUpdateRecord/schema/multi-arrays.avsc")));
         final String outputSchemaText = new String(Files.readAllBytes(Paths.get("src/test/resources/TestUpdateRecord/schema/multi-arrays.avsc")));
@@ -558,7 +554,6 @@ public class TestUpdateRecord {
     public void testUpdateComplexArrays() throws InitializationException, IOException {
         final JsonTreeReader jsonReader = new JsonTreeReader();
         runner.addControllerService("reader", jsonReader);
-        runner.setValidateExpressionUsage(false);
 
         final String inputSchemaText = new String(Files.readAllBytes(Paths.get("src/test/resources/TestUpdateRecord/schema/multi-arrays.avsc")));
         final String outputSchemaText = new String(Files.readAllBytes(Paths.get("src/test/resources/TestUpdateRecord/schema/multi-arrays.avsc")));

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java b/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java
index 880e49b..8d59c1c 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java
@@ -26,6 +26,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.reporting.InitializationException;
@@ -47,7 +48,7 @@ import java.util.concurrent.TimeUnit;
  */
 @Tags({ "dbcp", "jdbc", "database", "connection", "pooling", "store" })
 @CapabilityDescription("Provides Database Connection Pooling Service. Connections can be asked from pool and returned after usage.")
-@DynamicProperty(name = "JDBC property name", value = "JDBC property value", supportsExpressionLanguage = true,
+@DynamicProperty(name = "JDBC property name", value = "JDBC property value", expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
         description = "Specifies a property name and value to be set on the JDBC connection(s). "
                 + "If Expression Language is used, evaluation will be performed upon the controller service being enabled. "
                 + "Note that no flow file input (attributes, e.g.) is available for use in Expression Language constructs for these properties.")
@@ -60,7 +61,7 @@ public class DBCPConnectionPool extends AbstractControllerService implements DBC
         .defaultValue(null)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .build();
 
     public static final PropertyDescriptor DB_DRIVERNAME = new PropertyDescriptor.Builder()
@@ -69,7 +70,7 @@ public class DBCPConnectionPool extends AbstractControllerService implements DBC
         .defaultValue(null)
         .required(true)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .build();
 
     public static final PropertyDescriptor DB_DRIVER_LOCATION = new PropertyDescriptor.Builder()
@@ -79,7 +80,7 @@ public class DBCPConnectionPool extends AbstractControllerService implements DBC
         .defaultValue(null)
         .required(false)
         .addValidator(StandardValidators.createListValidator(true, true, StandardValidators.createURLorFileValidator()))
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .build();
 
     public static final PropertyDescriptor DB_USER = new PropertyDescriptor.Builder()
@@ -87,7 +88,7 @@ public class DBCPConnectionPool extends AbstractControllerService implements DBC
         .description("Database user name")
         .defaultValue(null)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .build();
 
     public static final PropertyDescriptor DB_PASSWORD = new PropertyDescriptor.Builder()
@@ -97,7 +98,7 @@ public class DBCPConnectionPool extends AbstractControllerService implements DBC
         .required(false)
         .sensitive(true)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .build();
 
     public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
@@ -128,7 +129,7 @@ public class DBCPConnectionPool extends AbstractControllerService implements DBC
             + "Note!! Using validation might have some performance penalty.")
         .required(false)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .build();
 
     private static final List<PropertyDescriptor> properties;
@@ -161,7 +162,7 @@ public class DBCPConnectionPool extends AbstractControllerService implements DBC
                 .required(false)
                 .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
                 .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
                 .dynamic(true)
                 .build();
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/HBaseClientService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/HBaseClientService.java b/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/HBaseClientService.java
index 357529e..cff3fb6 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/HBaseClientService.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/HBaseClientService.java
@@ -20,6 +20,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.hbase.put.PutColumn;
 import org.apache.nifi.hbase.put.PutFlowFile;
 import org.apache.nifi.hbase.scan.Column;
@@ -72,7 +73,7 @@ public interface HBaseClientService extends ControllerService {
             .name("Phoenix Client JAR Location")
             .description("The full path to the Phoenix client JAR. Required if Phoenix is installed on top of HBase.")
             .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .dynamicallyModifiesClasspath(true)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_1_1_2_ClientMapCacheService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_1_1_2_ClientMapCacheService.java b/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_1_1_2_ClientMapCacheService.java
index ca3a1db..7c10800 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_1_1_2_ClientMapCacheService.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_1_1_2_ClientMapCacheService.java
@@ -32,6 +32,7 @@ import org.apache.nifi.controller.ConfigurationContext;
 
 import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
 import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.distributed.cache.client.Deserializer;
 import java.io.ByteArrayOutputStream;
 import org.apache.nifi.reporting.InitializationException;
@@ -63,7 +64,7 @@ public class HBase_1_1_2_ClientMapCacheService extends AbstractControllerService
         .name("HBase Cache Table Name")
         .description("Name of the table on HBase to use for the cache.")
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .build();
 
@@ -71,7 +72,7 @@ public class HBase_1_1_2_ClientMapCacheService extends AbstractControllerService
         .name("HBase Column Family")
         .description("Name of the column family on HBase to use for the cache.")
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .defaultValue("f")
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .build();
@@ -81,7 +82,7 @@ public class HBase_1_1_2_ClientMapCacheService extends AbstractControllerService
         .description("Name of the column qualifier on HBase to use for the cache")
         .defaultValue("q")
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-services/nifi-http-context-map-bundle/nifi-http-context-map/src/main/java/org/apache/nifi/http/StandardHttpContextMap.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-http-context-map-bundle/nifi-http-context-map/src/main/java/org/apache/nifi/http/StandardHttpContextMap.java b/nifi-nar-bundles/nifi-standard-services/nifi-http-context-map-bundle/nifi-http-context-map/src/main/java/org/apache/nifi/http/StandardHttpContextMap.java
index ba70e0c..c25ddbf 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-http-context-map-bundle/nifi-http-context-map/src/main/java/org/apache/nifi/http/StandardHttpContextMap.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-http-context-map-bundle/nifi-http-context-map/src/main/java/org/apache/nifi/http/StandardHttpContextMap.java
@@ -40,6 +40,7 @@ import org.apache.nifi.annotation.lifecycle.OnShutdown;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.util.StandardValidators;
 
 @Tags({"http", "request", "response"})
@@ -61,7 +62,7 @@ public class StandardHttpContextMap extends AbstractControllerService implements
             .name("Request Expiration")
             .description("Specifies how long an HTTP Request should be left unanswered before being evicted from the cache and being responded to with a Service Unavailable status code")
             .required(true)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .defaultValue("1 min")
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-services/nifi-hwx-schema-registry-bundle/nifi-hwx-schema-registry-service/src/main/java/org/apache/nifi/schemaregistry/hortonworks/HortonworksSchemaRegistry.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-hwx-schema-registry-bundle/nifi-hwx-schema-registry-service/src/main/java/org/apache/nifi/schemaregistry/hortonworks/HortonworksSchemaRegistry.java b/nifi-nar-bundles/nifi-standard-services/nifi-hwx-schema-registry-bundle/nifi-hwx-schema-registry-service/src/main/java/org/apache/nifi/schemaregistry/hortonworks/HortonworksSchemaRegistry.java
index 88325ad..b33d5c8 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-hwx-schema-registry-bundle/nifi-hwx-schema-registry-service/src/main/java/org/apache/nifi/schemaregistry/hortonworks/HortonworksSchemaRegistry.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-hwx-schema-registry-bundle/nifi-hwx-schema-registry-service/src/main/java/org/apache/nifi/schemaregistry/hortonworks/HortonworksSchemaRegistry.java
@@ -32,6 +32,7 @@ import org.apache.nifi.avro.AvroTypeUtil;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.schema.access.SchemaField;
@@ -71,7 +72,7 @@ public class HortonworksSchemaRegistry extends AbstractControllerService impleme
         .displayName("Schema Registry URL")
         .description("URL of the schema registry that this Controller Service should connect to, including version. For example, http://localhost:9090/api/v1")
         .addValidator(StandardValidators.URL_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .required(true)
         .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/CSVRecordLookupService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/CSVRecordLookupService.java b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/CSVRecordLookupService.java
index 793ed54..8bbb141 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/CSVRecordLookupService.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/CSVRecordLookupService.java
@@ -27,6 +27,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.ControllerServiceInitializationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.reporting.InitializationException;
@@ -74,14 +75,14 @@ public class CSVRecordLookupService extends AbstractControllerService implements
                     .description("A CSV file that will serve as the data source.")
                     .required(true)
                     .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
-                    .expressionLanguageSupported(true)
+                    .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
                     .build();
 
     static final PropertyDescriptor CSV_FORMAT = new PropertyDescriptor.Builder()
             .name("csv-format")
             .displayName("CSV Format")
             .description("Specifies which \"format\" the CSV data is in.")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .allowableValues(Arrays.asList(CSVFormat.Predefined.values()).stream().map(e -> e.toString()).collect(Collectors.toSet()))
             .defaultValue(CSVFormat.Predefined.Default.toString())
             .required(true)
@@ -95,7 +96,7 @@ public class CSVRecordLookupService extends AbstractControllerService implements
                             "This is the field that will be matched against the property specified in the lookup processor.")
                     .required(true)
                     .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-                    .expressionLanguageSupported(true)
+                    .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
                     .build();
 
     public static final PropertyDescriptor IGNORE_DUPLICATES =
@@ -193,9 +194,9 @@ public class CSVRecordLookupService extends AbstractControllerService implements
 
     @OnEnabled
     public void onEnabled(final ConfigurationContext context) throws InitializationException, IOException {
-        this.csvFile = context.getProperty(CSV_FILE).getValue();
+        this.csvFile = context.getProperty(CSV_FILE).evaluateAttributeExpressions().getValue();
         this.csvFormat = CSVFormat.Predefined.valueOf(context.getProperty(CSV_FORMAT).getValue()).getFormat();
-        this.lookupKeyColumn = context.getProperty(LOOKUP_KEY_COLUMN).getValue();
+        this.lookupKeyColumn = context.getProperty(LOOKUP_KEY_COLUMN).evaluateAttributeExpressions().getValue();
         this.ignoreDuplicates = context.getProperty(IGNORE_DUPLICATES).asBoolean();
         this.watcher = new SynchronousFileWatcher(Paths.get(csvFile), new LastModifiedMonitor(), 30000L);
         try {

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/SimpleCsvFileLookupService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/SimpleCsvFileLookupService.java b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/SimpleCsvFileLookupService.java
index 878cbe8..7a2354a 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/SimpleCsvFileLookupService.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/SimpleCsvFileLookupService.java
@@ -44,6 +44,7 @@ import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ControllerServiceInitializationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.util.StandardValidators;
@@ -66,13 +67,13 @@ public class SimpleCsvFileLookupService extends AbstractControllerService implem
             .description("A CSV file.")
             .required(true)
             .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     static final PropertyDescriptor CSV_FORMAT = new PropertyDescriptor.Builder()
         .name("CSV Format")
         .description("Specifies which \"format\" the CSV data is in, or specifies if custom formatting should be used.")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues(Arrays.asList(CSVFormat.Predefined.values()).stream().map(e -> e.toString()).collect(Collectors.toSet()))
         .defaultValue(CSVFormat.Predefined.Default.toString())
         .required(true)
@@ -85,7 +86,7 @@ public class SimpleCsvFileLookupService extends AbstractControllerService implem
             .description("Lookup key column.")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor LOOKUP_VALUE_COLUMN =
@@ -95,7 +96,7 @@ public class SimpleCsvFileLookupService extends AbstractControllerService implem
             .description("Lookup value column.")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor IGNORE_DUPLICATES =
@@ -180,10 +181,10 @@ public class SimpleCsvFileLookupService extends AbstractControllerService implem
 
     @OnEnabled
     public void onEnabled(final ConfigurationContext context) throws InitializationException, IOException, FileNotFoundException {
-        this.csvFile = context.getProperty(CSV_FILE).getValue();
+        this.csvFile = context.getProperty(CSV_FILE).evaluateAttributeExpressions().getValue();
         this.csvFormat = CSVFormat.Predefined.valueOf(context.getProperty(CSV_FORMAT).getValue()).getFormat();
-        this.lookupKeyColumn = context.getProperty(LOOKUP_KEY_COLUMN).getValue();
-        this.lookupValueColumn = context.getProperty(LOOKUP_VALUE_COLUMN).getValue();
+        this.lookupKeyColumn = context.getProperty(LOOKUP_KEY_COLUMN).evaluateAttributeExpressions().getValue();
+        this.lookupValueColumn = context.getProperty(LOOKUP_VALUE_COLUMN).evaluateAttributeExpressions().getValue();
         this.ignoreDuplicates = context.getProperty(IGNORE_DUPLICATES).asBoolean();
         this.watcher = new SynchronousFileWatcher(Paths.get(csvFile), new LastModifiedMonitor(), 30000L);
         try {

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/SimpleKeyValueLookupService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/SimpleKeyValueLookupService.java b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/SimpleKeyValueLookupService.java
index ec9371d..2f99c36 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/SimpleKeyValueLookupService.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/SimpleKeyValueLookupService.java
@@ -31,6 +31,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.Validator;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 
 @Tags({"lookup", "enrich", "key", "value"})
 @CapabilityDescription("Allows users to add key/value pairs as User-defined Properties. Each property that is added can be looked up by Property Name. "
@@ -47,7 +48,7 @@ public class SimpleKeyValueLookupService extends AbstractControllerService imple
             .required(false)
             .dynamic(true)
             .addValidator(Validator.VALID)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/configuration2/CommonsConfigurationLookupService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/configuration2/CommonsConfigurationLookupService.java b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/configuration2/CommonsConfigurationLookupService.java
index 9248132..3957f80 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/configuration2/CommonsConfigurationLookupService.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/configuration2/CommonsConfigurationLookupService.java
@@ -41,6 +41,7 @@ import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ControllerServiceInitializationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.lookup.LookupFailureException;
 import org.apache.nifi.lookup.StringLookupService;
@@ -65,7 +66,7 @@ public abstract class CommonsConfigurationLookupService<T extends FileBasedConfi
             .description("A configuration file")
             .required(true)
             .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     private final Class<T> resultClass = (Class<T>) ((ParameterizedType) getClass().getGenericSuperclass()).getActualTypeArguments()[0];
@@ -99,7 +100,7 @@ public abstract class CommonsConfigurationLookupService<T extends FileBasedConfi
 
     @OnEnabled
     public void onEnabled(final ConfigurationContext context) throws InitializationException {
-        final String config = context.getProperty(CONFIGURATION_FILE).getValue();
+        final String config = context.getProperty(CONFIGURATION_FILE).evaluateAttributeExpressions().getValue();
         final FileBasedBuilderParameters params = new Parameters().fileBased().setFile(new File(config));
         this.builder = new ReloadingFileBasedConfigurationBuilder<>(resultClass).configure(params);
         builder.addEventListener(ConfigurationBuilderEvent.CONFIGURATION_REQUEST,

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/maxmind/IPLookupService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/maxmind/IPLookupService.java b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/maxmind/IPLookupService.java
index 5661b94..b2fb463 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/maxmind/IPLookupService.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/maxmind/IPLookupService.java
@@ -42,6 +42,7 @@ import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.lookup.LookupFailureException;
 import org.apache.nifi.lookup.RecordLookupService;
 import org.apache.nifi.processor.util.StandardValidators;
@@ -87,7 +88,7 @@ public class IPLookupService extends AbstractControllerService implements Record
         .description("Path to Maxmind IP Enrichment Database File")
         .required(true)
         .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .build();
     static final PropertyDescriptor LOOKUP_CITY = new PropertyDescriptor.Builder()
         .name("lookup-city")
@@ -95,14 +96,14 @@ public class IPLookupService extends AbstractControllerService implements Record
         .description("Specifies whether or not information about the geographic information, such as cities, corresponding to the IP address should be returned")
         .allowableValues("true", "false")
         .defaultValue("true")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(true)
         .build();
     static final PropertyDescriptor LOOKUP_ISP = new PropertyDescriptor.Builder()
         .name("lookup-isp")
         .displayName("Lookup ISP")
         .description("Specifies whether or not information about the Information Service Provider corresponding to the IP address should be returned")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues("true", "false")
         .defaultValue("false")
         .required(true)
@@ -112,7 +113,7 @@ public class IPLookupService extends AbstractControllerService implements Record
         .displayName("Lookup Domain Name")
         .description("Specifies whether or not information about the Domain Name corresponding to the IP address should be returned. "
             + "If true, the lookup will contain second-level domain information, such as foo.com but will not contain bar.foo.com")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues("true", "false")
         .defaultValue("false")
         .required(true)
@@ -122,7 +123,7 @@ public class IPLookupService extends AbstractControllerService implements Record
         .displayName("Lookup Connection Type")
         .description("Specifies whether or not information about the Connection Type corresponding to the IP address should be returned. "
             + "If true, the lookup will contain a 'connectionType' field that (if populated) will contain a value of 'Dialup', 'Cable/DSL', 'Corporate', or 'Cellular'")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues("true", "false")
         .defaultValue("false")
         .required(true)
@@ -131,7 +132,7 @@ public class IPLookupService extends AbstractControllerService implements Record
         .name("lookup-anonymous-ip")
         .displayName("Lookup Anonymous IP Information")
         .description("Specifies whether or not information about whether or not the IP address belongs to an anonymous network should be returned.")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues("true", "false")
         .defaultValue("false")
         .required(true)

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java
index 9f133a6..858ef70 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java
@@ -32,6 +32,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.schema.access.SchemaAccessStrategy;
 import org.apache.nifi.schema.access.SchemaAccessUtils;
@@ -67,7 +68,7 @@ public class CSVReader extends SchemaRegistryService implements RecordReaderFact
             .displayName("CSV Parser")
             .description("Specifies which parser to use to read CSV records. NOTE: Different parsers may support different subsets of functionality "
                     + "and may also exhibit different levels of performance.")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .allowableValues(APACHE_COMMONS_CSV, JACKSON_CSV)
             .defaultValue(APACHE_COMMONS_CSV.getValue())
             .required(true)

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java
index 6eea8e3..29e9636 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java
@@ -35,6 +35,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.schema.access.SchemaAccessStrategy;
@@ -86,7 +87,7 @@ public class GrokReader extends SchemaRegistryService implements RecordReaderFac
             + "will be used. If specified, all patterns in the given pattern file will override the default patterns. See the Controller Service's "
             + "Additional Details for a list of pre-defined patterns.")
         .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .required(false)
         .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathReader.java
index cab4449..f8f1ea2 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathReader.java
@@ -35,6 +35,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.schema.access.SchemaNotFoundException;
 import org.apache.nifi.serialization.DateTimeUtils;
@@ -56,7 +57,7 @@ import com.jayway.jsonpath.JsonPath;
     value="A JSONPath Expression that will be evaluated against each JSON record. The result of the JSONPath will be the value of the "
         + "field whose name is the same as the property name.",
     description="User-defined properties identifiy how to extract specific fields from a JSON object in order to create a Record",
-    supportsExpressionLanguage=false)
+    expressionLanguageScope=ExpressionLanguageScope.NONE)
 public class JsonPathReader extends SchemaRegistryService implements RecordReaderFactory {
 
     private volatile String dateFormat;


[10/15] nifi git commit: NIFI-4149 - Indicate if EL is evaluated against FFs or not - take into account input requirement for documentation rendering - Renamed variable registry scope and added comments - Doc + change in mock framework to check scope + u

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestQueryElasticsearchHttp.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestQueryElasticsearchHttp.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestQueryElasticsearchHttp.java
index 4789496..1b07d22 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestQueryElasticsearchHttp.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestQueryElasticsearchHttp.java
@@ -62,7 +62,6 @@ public class TestQueryElasticsearchHttp {
     @Test
     public void testQueryElasticsearchOnTrigger_withInput() throws IOException {
         runner = TestRunners.newTestRunner(new QueryElasticsearchHttpTestProcessor());
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
@@ -81,7 +80,6 @@ public class TestQueryElasticsearchHttp {
     @Test
     public void testQueryElasticsearchOnTrigger_withInput_EL() throws IOException {
         runner = TestRunners.newTestRunner(new QueryElasticsearchHttpTestProcessor());
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "${es.url}");
 
         runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
@@ -104,7 +102,6 @@ public class TestQueryElasticsearchHttp {
     @Test
     public void testQueryElasticsearchOnTrigger_withInput_attributeTarget() throws IOException {
         runner = TestRunners.newTestRunner(new QueryElasticsearchHttpTestProcessor());
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
@@ -130,7 +127,6 @@ public class TestQueryElasticsearchHttp {
     @Test
     public void testQueryElasticsearchOnTrigger_withNoInput() throws IOException {
         runner = TestRunners.newTestRunner(new QueryElasticsearchHttpTestProcessor());
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
@@ -175,7 +171,6 @@ public class TestQueryElasticsearchHttp {
     @Test
     public void testQueryElasticsearchOnTriggerWithFields() throws IOException {
         runner = TestRunners.newTestRunner(new QueryElasticsearchHttpTestProcessor());
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
@@ -195,7 +190,6 @@ public class TestQueryElasticsearchHttp {
     @Test
     public void testQueryElasticsearchOnTriggerWithLimit() throws IOException {
         runner = TestRunners.newTestRunner(new QueryElasticsearchHttpTestProcessor());
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
@@ -221,7 +215,6 @@ public class TestQueryElasticsearchHttp {
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
         runner.setProperty(QueryElasticsearchHttp.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(QueryElasticsearchHttp.QUERY, "${doc_id}");
 
         runner.enqueue("".getBytes(), new HashMap<String, String>() {
@@ -247,7 +240,6 @@ public class TestQueryElasticsearchHttp {
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
         runner.setProperty(QueryElasticsearchHttp.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(QueryElasticsearchHttp.QUERY, "${doc_id}");
 
         runner.enqueue("".getBytes(), new HashMap<String, String>() {
@@ -273,7 +265,6 @@ public class TestQueryElasticsearchHttp {
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
         runner.setProperty(QueryElasticsearchHttp.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(QueryElasticsearchHttp.QUERY, "${doc_id}");
 
         runner.enqueue("".getBytes(), new HashMap<String, String>() {
@@ -299,7 +290,6 @@ public class TestQueryElasticsearchHttp {
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
         runner.setProperty(QueryElasticsearchHttp.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(QueryElasticsearchHttp.QUERY, "${doc_id}");
 
         runner.enqueue("".getBytes(), new HashMap<String, String>() {
@@ -326,7 +316,6 @@ public class TestQueryElasticsearchHttp {
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
         runner.setProperty(QueryElasticsearchHttp.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(QueryElasticsearchHttp.QUERY, "${doc_id}");
 
         runner.setIncomingConnection(false);
@@ -348,7 +337,6 @@ public class TestQueryElasticsearchHttp {
         runner.setProperty(QueryElasticsearchHttp.PROP_SSL_CONTEXT_SERVICE, "ssl-context");
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(QueryElasticsearchHttp.QUERY, "${doc_id}");
 
         // Allow time for the controller service to fully initialize
@@ -367,7 +355,6 @@ public class TestQueryElasticsearchHttp {
         QueryElasticsearchHttpTestProcessor p = new QueryElasticsearchHttpTestProcessor();
         p.setExpectedParam("myparam=myvalue");
         runner = TestRunners.newTestRunner(p);
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
@@ -480,6 +467,7 @@ public class TestQueryElasticsearchHttp {
             });
         }
 
+        @Override
         protected OkHttpClient getClient() {
             return client;
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestScrollElasticsearchHttp.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestScrollElasticsearchHttp.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestScrollElasticsearchHttp.java
index 1e687f1..f22275a 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestScrollElasticsearchHttp.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestScrollElasticsearchHttp.java
@@ -63,7 +63,6 @@ public class TestScrollElasticsearchHttp {
     @Test
     public void testScrollElasticsearchOnTrigger_withNoInput() throws IOException {
         runner = TestRunners.newTestRunner(new ScrollElasticsearchHttpTestProcessor());
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(ScrollElasticsearchHttp.INDEX, "doc");
@@ -83,7 +82,6 @@ public class TestScrollElasticsearchHttp {
     @Test
     public void testScrollElasticsearchOnTrigger_withNoInput_EL() throws IOException {
         runner = TestRunners.newTestRunner(new ScrollElasticsearchHttpTestProcessor());
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "${es.url}");
 
         runner.setProperty(ScrollElasticsearchHttp.INDEX, "doc");
@@ -131,7 +129,6 @@ public class TestScrollElasticsearchHttp {
     @Test
     public void testScrollElasticsearchOnTriggerWithFields() throws IOException {
         runner = TestRunners.newTestRunner(new ScrollElasticsearchHttpTestProcessor());
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(ScrollElasticsearchHttp.INDEX, "doc");
@@ -157,7 +154,6 @@ public class TestScrollElasticsearchHttp {
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(ScrollElasticsearchHttp.INDEX, "doc");
         runner.setProperty(ScrollElasticsearchHttp.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(ScrollElasticsearchHttp.QUERY, "${doc_id}");
         runner.setIncomingConnection(false);
 
@@ -182,7 +178,6 @@ public class TestScrollElasticsearchHttp {
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(ScrollElasticsearchHttp.INDEX, "doc");
         runner.setProperty(ScrollElasticsearchHttp.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(ScrollElasticsearchHttp.QUERY, "${doc_id}");
         runner.setIncomingConnection(false);
 
@@ -207,7 +202,6 @@ public class TestScrollElasticsearchHttp {
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(ScrollElasticsearchHttp.INDEX, "doc");
         runner.setProperty(ScrollElasticsearchHttp.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(ScrollElasticsearchHttp.QUERY, "${doc_id}");
 
         runner.enqueue("".getBytes(), new HashMap<String, String>() {
@@ -232,7 +226,6 @@ public class TestScrollElasticsearchHttp {
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(ScrollElasticsearchHttp.INDEX, "doc");
         runner.setProperty(ScrollElasticsearchHttp.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(ScrollElasticsearchHttp.QUERY, "${doc_id}");
 
         runner.setIncomingConnection(false);
@@ -254,7 +247,6 @@ public class TestScrollElasticsearchHttp {
         runner.setProperty(ScrollElasticsearchHttp.PROP_SSL_CONTEXT_SERVICE, "ssl-context");
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(ScrollElasticsearchHttp.INDEX, "doc");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(ScrollElasticsearchHttp.QUERY, "${doc_id}");
         runner.setIncomingConnection(false);
 
@@ -278,7 +270,6 @@ public class TestScrollElasticsearchHttp {
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(ScrollElasticsearchHttp.INDEX, "doc");
         runner.setProperty(ScrollElasticsearchHttp.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(ScrollElasticsearchHttp.QUERY, "${doc_id}");
 
         runner.enqueue("".getBytes(), new HashMap<String, String>() {
@@ -302,7 +293,6 @@ public class TestScrollElasticsearchHttp {
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(ScrollElasticsearchHttp.INDEX, "doc");
         runner.setProperty(ScrollElasticsearchHttp.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(ScrollElasticsearchHttp.QUERY, "${doc_id}");
 
         runner.enqueue("".getBytes(), new HashMap<String, String>() {
@@ -323,7 +313,6 @@ public class TestScrollElasticsearchHttp {
         ScrollElasticsearchHttpTestProcessor p = new ScrollElasticsearchHttpTestProcessor();
         p.setExpectedParam("myparam=myvalue");
         runner = TestRunners.newTestRunner(p);
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(ScrollElasticsearchHttp.INDEX, "doc");
@@ -438,6 +427,7 @@ public class TestScrollElasticsearchHttp {
             });
         }
 
+        @Override
         protected OkHttpClient getClient() {
             return client;
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/AbstractEmailProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/AbstractEmailProcessor.java b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/AbstractEmailProcessor.java
index 8e6dbae..9b6d396 100644
--- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/AbstractEmailProcessor.java
+++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/AbstractEmailProcessor.java
@@ -36,6 +36,7 @@ import javax.mail.MessagingException;
 
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -62,7 +63,7 @@ abstract class AbstractEmailProcessor<T extends AbstractMailReceiver> extends Ab
             .displayName("Host Name")
             .description("Network address of Email server (e.g., pop.gmail.com, imap.gmail.com . . .)")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
     public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
@@ -70,7 +71,7 @@ abstract class AbstractEmailProcessor<T extends AbstractMailReceiver> extends Ab
             .displayName("Port")
             .description("Numeric value identifying Port of Email server (e.g., 993)")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.PORT_VALIDATOR)
             .build();
     public static final PropertyDescriptor USER = new PropertyDescriptor.Builder()
@@ -78,7 +79,7 @@ abstract class AbstractEmailProcessor<T extends AbstractMailReceiver> extends Ab
             .displayName("User Name")
             .description("User Name used for authentication and authorization with Email server.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
     public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
@@ -86,7 +87,7 @@ abstract class AbstractEmailProcessor<T extends AbstractMailReceiver> extends Ab
             .displayName("Password")
             .description("Password used for authentication and authorization with Email server.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .sensitive(true)
             .build();
@@ -95,7 +96,7 @@ abstract class AbstractEmailProcessor<T extends AbstractMailReceiver> extends Ab
             .displayName("Folder")
             .description("Email folder to retrieve messages from (e.g., INBOX)")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue("INBOX")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
@@ -104,7 +105,7 @@ abstract class AbstractEmailProcessor<T extends AbstractMailReceiver> extends Ab
             .displayName("Fetch Size")
             .description("Specify the maximum number of Messages to fetch per call to Email Server.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue("10")
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
             .build();
@@ -123,7 +124,7 @@ abstract class AbstractEmailProcessor<T extends AbstractMailReceiver> extends Ab
             .description("The amount of time to wait to connect to Email server")
             .required(true)
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue("30 sec")
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ConsumeEWS.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ConsumeEWS.java b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ConsumeEWS.java
index 70a26dd..462f405 100644
--- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ConsumeEWS.java
+++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ConsumeEWS.java
@@ -52,6 +52,7 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -92,7 +93,7 @@ public class ConsumeEWS extends AbstractProcessor {
             .displayName("User Name")
             .description("User Name used for authentication and authorization with Email server.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
     public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
@@ -100,7 +101,7 @@ public class ConsumeEWS extends AbstractProcessor {
             .displayName("Password")
             .description("Password used for authentication and authorization with Email server.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .sensitive(true)
             .build();
@@ -109,7 +110,7 @@ public class ConsumeEWS extends AbstractProcessor {
             .displayName("Folder")
             .description("Email folder to retrieve messages from (e.g., INBOX)")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue("INBOX")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
@@ -118,7 +119,7 @@ public class ConsumeEWS extends AbstractProcessor {
             .displayName("Fetch Size")
             .description("Specify the maximum number of Messages to fetch per call to Email Server.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue("10")
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
             .build();
@@ -137,7 +138,7 @@ public class ConsumeEWS extends AbstractProcessor {
             .description("The amount of time to wait to connect to Email server")
             .required(true)
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue("30 sec")
             .build();
     public static final PropertyDescriptor EXCHANGE_VERSION = new PropertyDescriptor.Builder()
@@ -256,7 +257,7 @@ public class ConsumeEWS extends AbstractProcessor {
             this.messageQueue = new ArrayBlockingQueue<>(fetchSize);
         }
 
-        this.folderName = context.getProperty(FOLDER).getValue();
+        this.folderName = context.getProperty(FOLDER).evaluateAttributeExpressions().getValue();
 
         Message emailMessage = this.receiveMessage(context);
         if (emailMessage != null) {
@@ -274,8 +275,8 @@ public class ConsumeEWS extends AbstractProcessor {
         final String timeoutInMillis = String.valueOf(context.getProperty(CONNECTION_TIMEOUT).evaluateAttributeExpressions().asTimePeriod(TimeUnit.MILLISECONDS));
         service.setTimeout(Integer.parseInt(timeoutInMillis));
 
-        String userEmail = context.getProperty(USER).getValue();
-        String password = context.getProperty(PASSWORD).getValue();
+        String userEmail = context.getProperty(USER).evaluateAttributeExpressions().getValue();
+        String password = context.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
 
         ExchangeCredentials credentials = new WebCredentials(userEmail, password);
         service.setCredentials(credentials);

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java
index 22936fd..5a11f04 100644
--- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java
+++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java
@@ -30,6 +30,7 @@ import org.apache.nifi.annotation.behavior.WritesAttributes;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
@@ -98,7 +99,7 @@ public class ExtractEmailHeaders extends AbstractProcessor {
                     "NOTE the header key is case insensitive and will be matched as lower-case." +
                     " Values will respect email contents.")
             .required(false)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .defaultValue("x-mailer")
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
index 62d9bd4..bed09b1 100644
--- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
+++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
@@ -41,6 +41,7 @@ import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
 import org.apache.nifi.processor.DataUnit;
@@ -85,7 +86,7 @@ public class ListenSMTP extends AbstractSessionFactoryProcessor {
                     + "NOTE that on Unix derivative operating  systems this port must "
                     + "be higher than 1024 unless NiFi is running as with root user permissions.")
             .required(true)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .addValidator(StandardValidators.PORT_VALIDATOR)
             .build();
 
@@ -95,7 +96,7 @@ public class ListenSMTP extends AbstractSessionFactoryProcessor {
             .description("The maximum number of simultaneous SMTP connections.")
             .required(true)
             .defaultValue("1")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .addValidator(StandardValidators.INTEGER_VALIDATOR)
             .build();
 
@@ -105,7 +106,7 @@ public class ListenSMTP extends AbstractSessionFactoryProcessor {
             .description("The maximum time to wait for an action of SMTP client.")
             .defaultValue("60 seconds")
             .required(true)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
             .build();
 
@@ -115,7 +116,7 @@ public class ListenSMTP extends AbstractSessionFactoryProcessor {
             .description("The maximum number of bytes the server will accept.")
             .required(true)
             .defaultValue("20 MB")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .addValidator(StandardValidators.createDataSizeBoundsValidator(1, Integer.MAX_VALUE))
             .build();
 
@@ -141,7 +142,7 @@ public class ListenSMTP extends AbstractSessionFactoryProcessor {
             .displayName("SMTP hostname")
             .description("The hostname to be embedded into the banner displayed when an "
                     + "SMTP client connects to the processor TCP port .")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-processors/src/main/java/org/apache/nifi/processors/AbstractEnrichIP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-processors/src/main/java/org/apache/nifi/processors/AbstractEnrichIP.java b/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-processors/src/main/java/org/apache/nifi/processors/AbstractEnrichIP.java
index f55e701..30226a9 100644
--- a/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-processors/src/main/java/org/apache/nifi/processors/AbstractEnrichIP.java
+++ b/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-processors/src/main/java/org/apache/nifi/processors/AbstractEnrichIP.java
@@ -31,6 +31,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessorInitializationContext;
@@ -56,7 +57,7 @@ public abstract class AbstractEnrichIP extends AbstractProcessor {
             .displayName("IP Address Attribute")
             .required(true)
             .description("The name of an attribute whose value is a dotted decimal IP address for which enrichment should occur")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING))
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-processors/src/main/java/org/apache/nifi/processors/enrich/AbstractEnrichProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-processors/src/main/java/org/apache/nifi/processors/enrich/AbstractEnrichProcessor.java b/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-processors/src/main/java/org/apache/nifi/processors/enrich/AbstractEnrichProcessor.java
index 0ba7934..e0307a0 100644
--- a/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-processors/src/main/java/org/apache/nifi/processors/enrich/AbstractEnrichProcessor.java
+++ b/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-processors/src/main/java/org/apache/nifi/processors/enrich/AbstractEnrichProcessor.java
@@ -26,6 +26,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.util.StandardValidators;
@@ -44,7 +45,7 @@ public abstract class AbstractEnrichProcessor extends AbstractProcessor {
             .displayName("Lookup value")
             .required(true)
             .description("The value that should be used to populate the query")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -70,7 +71,7 @@ public abstract class AbstractEnrichProcessor extends AbstractProcessor {
             .description("Choice between a splitter and regex matcher used to parse the results of the query into attribute groups.\n" +
             "NOTE: This is a multiline regular expression, therefore, the DFM should decide how to handle trailing new line " +
             "characters.")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(false)
             .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/src/main/java/org/apache/nifi/hadoop/KerberosProperties.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/src/main/java/org/apache/nifi/hadoop/KerberosProperties.java b/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/src/main/java/org/apache/nifi/hadoop/KerberosProperties.java
index 676c684..7d0210f 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/src/main/java/org/apache/nifi/hadoop/KerberosProperties.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/src/main/java/org/apache/nifi/hadoop/KerberosProperties.java
@@ -21,6 +21,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.util.StandardValidators;
 
@@ -85,7 +86,7 @@ public class KerberosProperties {
                 .description("Kerberos principal to authenticate as. Requires nifi.kerberos.krb5.file to be set in your nifi.properties")
                 .addValidator(kerberosConfigValidator)
                 .addValidator(StandardValidators.ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
                 .build();
 
         this.kerberosKeytab = new PropertyDescriptor.Builder()
@@ -94,7 +95,7 @@ public class KerberosProperties {
                 .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
                 .addValidator(kerberosConfigValidator)
                 .addValidator(StandardValidators.ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
                 .build();
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java b/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
index 54ddc66..1eca5af 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
@@ -30,6 +30,7 @@ import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.hadoop.KerberosProperties;
 import org.apache.nifi.hadoop.SecurityUtil;
 import org.apache.nifi.kerberos.KerberosCredentialsService;
@@ -78,7 +79,7 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
                     + "To use swebhdfs, see 'Additional Details' section of PutHDFS's documentation.")
             .required(false)
             .addValidator(HadoopValidators.ONE_OR_MORE_FILE_EXISTS_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder()
@@ -86,7 +87,7 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
             .description("The HDFS directory from which files should be read")
             .required(true)
             .addValidator(StandardValidators.ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor COMPRESSION_CODEC = new PropertyDescriptor.Builder()
@@ -103,7 +104,7 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
             .defaultValue("4 hours")
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor ADDITIONAL_CLASSPATH_RESOURCES = new PropertyDescriptor.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/listen/AbstractListenEventBatchingProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/listen/AbstractListenEventBatchingProcessor.java b/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/listen/AbstractListenEventBatchingProcessor.java
index 9a97671..c83fdae 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/listen/AbstractListenEventBatchingProcessor.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/listen/AbstractListenEventBatchingProcessor.java
@@ -20,6 +20,7 @@ import static org.apache.nifi.processor.util.listen.ListenerProperties.NETWORK_I
 
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -54,7 +55,7 @@ public abstract class AbstractListenEventBatchingProcessor<E extends Event> exte
                     "The maximum number of messages to add to a single FlowFile. If multiple messages are available, they will be concatenated along with "
                             + "the <Message Delimiter> up to this configured maximum number of messages")
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .defaultValue("1")
             .required(true)
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/listen/ListenerProperties.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/listen/ListenerProperties.java b/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/listen/ListenerProperties.java
index 5e4c639..128a9d9 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/listen/ListenerProperties.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/listen/ListenerProperties.java
@@ -21,6 +21,7 @@ import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
 import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 
 import java.net.NetworkInterface;
 import java.net.SocketException;
@@ -81,7 +82,7 @@ public class ListenerProperties {
                     return result;
                 }
             })
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/put/AbstractPutEventProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/put/AbstractPutEventProcessor.java b/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/put/AbstractPutEventProcessor.java
index d09fe06..a5ba8ba 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/put/AbstractPutEventProcessor.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/put/AbstractPutEventProcessor.java
@@ -20,6 +20,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -56,14 +57,14 @@ public abstract class AbstractPutEventProcessor extends AbstractSessionFactoryPr
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .defaultValue("localhost")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
     public static final PropertyDescriptor PORT = new PropertyDescriptor
             .Builder().name("Port")
             .description("The port on the destination.")
             .required(true)
             .addValidator(StandardValidators.PORT_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
     public static final PropertyDescriptor MAX_SOCKET_SEND_BUFFER_SIZE = new PropertyDescriptor.Builder()
             .name("Max Size of Socket Send Buffer")
@@ -106,7 +107,7 @@ public abstract class AbstractPutEventProcessor extends AbstractSessionFactoryPr
                     + "relationship.")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     public static final PropertyDescriptor CHARSET = new PropertyDescriptor.Builder()
             .name("Character Set")
@@ -130,7 +131,7 @@ public abstract class AbstractPutEventProcessor extends AbstractSessionFactoryPr
                     + "enter '\\n'. For a tab character use '\\t'. Finally for a carriage return use '\\r'.")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     public static final PropertyDescriptor CONNECTION_PER_FLOWFILE = new PropertyDescriptor.Builder()
             .name("Connection Per FlowFile")

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/schema/access/SchemaAccessUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/schema/access/SchemaAccessUtils.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/schema/access/SchemaAccessUtils.java
index 17aef2a..111b02a 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/schema/access/SchemaAccessUtils.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/schema/access/SchemaAccessUtils.java
@@ -23,6 +23,7 @@ import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.schemaregistry.services.SchemaRegistry;
 
@@ -73,7 +74,7 @@ public class SchemaAccessUtils {
             .displayName("Schema Name")
             .description("Specifies the name of the schema to lookup in the Schema Registry property")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("${schema.name}")
             .required(false)
             .build();
@@ -103,7 +104,7 @@ public class SchemaAccessUtils {
             .displayName("Schema Text")
             .description("The text of an Avro-formatted Schema")
             .addValidator(new AvroSchemaValidator())
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("${avro.schema}")
             .required(false)
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-hadoop-record-utils/src/main/java/org/apache/nifi/processors/hadoop/AbstractFetchHDFSRecord.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-hadoop-record-utils/src/main/java/org/apache/nifi/processors/hadoop/AbstractFetchHDFSRecord.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-hadoop-record-utils/src/main/java/org/apache/nifi/processors/hadoop/AbstractFetchHDFSRecord.java
index 96631ef..41b0365 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-hadoop-record-utils/src/main/java/org/apache/nifi/processors/hadoop/AbstractFetchHDFSRecord.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-hadoop-record-utils/src/main/java/org/apache/nifi/processors/hadoop/AbstractFetchHDFSRecord.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
 import org.apache.nifi.annotation.configuration.DefaultSettings;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.ProcessContext;
@@ -68,7 +69,7 @@ public abstract class AbstractFetchHDFSRecord extends AbstractHadoopProcessor {
             .displayName("Filename")
             .description("The name of the file to retrieve")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("${path}/${filename}")
             .addValidator(StandardValidators.ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR)
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/csv/CSVUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/csv/CSVUtils.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/csv/CSVUtils.java
index eecf290..f379bea 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/csv/CSVUtils.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/csv/CSVUtils.java
@@ -24,6 +24,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.util.StandardValidators;
 
 public class CSVUtils {
@@ -41,7 +42,7 @@ public class CSVUtils {
     public static final PropertyDescriptor CSV_FORMAT = new PropertyDescriptor.Builder()
         .name("CSV Format")
         .description("Specifies which \"format\" the CSV data is in, or specifies if custom formatting should be used.")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues(CUSTOM, RFC_4180, EXCEL, TDF, MYSQL, INFORMIX_UNLOAD, INFORMIX_UNLOAD_CSV)
         .defaultValue(CUSTOM.getValue())
         .required(true)
@@ -50,7 +51,7 @@ public class CSVUtils {
         .name("Value Separator")
         .description("The character that is used to separate values/fields in a CSV Record")
         .addValidator(CSVValidators.UNESCAPED_SINGLE_CHAR_VALIDATOR)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .defaultValue(",")
         .required(true)
         .build();
@@ -58,7 +59,7 @@ public class CSVUtils {
         .name("Quote Character")
         .description("The character that is used to quote values so that escape characters do not have to be used")
         .addValidator(new CSVValidators.SingleCharacterValidator())
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .defaultValue("\"")
         .required(true)
         .build();
@@ -70,7 +71,7 @@ public class CSVUtils {
             + "present and won't be processed as a Record. Otherwise, if 'true', then the first line of CSV data will not be processed as a record and if 'false',"
             + "then the first line will be interpreted as a record.")
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues("true", "false")
         .defaultValue("false")
         .required(true)
@@ -82,7 +83,7 @@ public class CSVUtils {
             + "the Reader will interpret the fields. If this property is true, then the field names mapped to each column are driven only by the configured schema and "
             + "any fields not in the schema will be ignored. If this property is false, then the field names found in the CSV Header will be used as the names of the "
             + "fields.")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues("true", "false")
         .defaultValue("false")
         .required(false)
@@ -91,14 +92,14 @@ public class CSVUtils {
         .name("Comment Marker")
         .description("The character that is used to denote the start of a comment. Any line that begins with this comment will be ignored.")
         .addValidator(new CSVValidators.SingleCharacterValidator())
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(false)
         .build();
     public static final PropertyDescriptor ESCAPE_CHAR = new PropertyDescriptor.Builder()
         .name("Escape Character")
         .description("The character that is used to escape characters that would otherwise have a specific meaning to the CSV Parser.")
         .addValidator(new CSVValidators.SingleCharacterValidator())
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .defaultValue("\\")
         .required(true)
         .build();
@@ -106,13 +107,13 @@ public class CSVUtils {
         .name("Null String")
         .description("Specifies a String that, if present as a value in the CSV, should be considered a null field instead of using the literal value.")
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(false)
         .build();
     public static final PropertyDescriptor TRIM_FIELDS = new PropertyDescriptor.Builder()
         .name("Trim Fields")
         .description("Whether or not white space should be removed from the beginning and end of fields")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues("true", "false")
         .defaultValue("true")
         .required(true)
@@ -121,7 +122,7 @@ public class CSVUtils {
         .name("csvutils-character-set")
         .displayName("Character Set")
         .description("The Character Encoding that is used to encode/decode the CSV file")
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
         .defaultValue("UTF-8")
         .required(true)
@@ -138,7 +139,7 @@ public class CSVUtils {
     public static final PropertyDescriptor QUOTE_MODE = new PropertyDescriptor.Builder()
         .name("Quote Mode")
         .description("Specifies how fields should be quoted when they are written")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues(QUOTE_ALL, QUOTE_MINIMAL, QUOTE_NON_NUMERIC, QUOTE_NONE)
         .defaultValue(QUOTE_MINIMAL.getValue())
         .required(true)
@@ -146,7 +147,7 @@ public class CSVUtils {
     public static final PropertyDescriptor TRAILING_DELIMITER = new PropertyDescriptor.Builder()
         .name("Include Trailing Delimiter")
         .description("If true, a trailing delimiter will be added to each CSV Record that is written. If false, the trailing delimiter will be omitted.")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues("true", "false")
         .defaultValue("false")
         .required(true)
@@ -155,7 +156,7 @@ public class CSVUtils {
         .name("Record Separator")
         .description("Specifies the characters to use in order to separate CSV Records")
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .defaultValue("\\n")
         .required(true)
         .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/serialization/DateTimeUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/serialization/DateTimeUtils.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/serialization/DateTimeUtils.java
index 6336943..45b3f93 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/serialization/DateTimeUtils.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/serialization/DateTimeUtils.java
@@ -18,6 +18,7 @@
 package org.apache.nifi.serialization;
 
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 
 public class DateTimeUtils {
     public static final PropertyDescriptor DATE_FORMAT = new PropertyDescriptor.Builder()
@@ -26,7 +27,7 @@ public class DateTimeUtils {
             + "If not specified, Date fields will be assumed to be number of milliseconds since epoch (Midnight, Jan 1, 1970 GMT). "
             + "If specified, the value must match the Java Simple Date Format (for example, MM/dd/yyyy for a two-digit month, followed by "
             + "a two-digit day, followed by a four-digit year, all separated by '/' characters, as in 01/01/2017).")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .addValidator(new SimpleDateFormatValidator())
         .required(false)
         .build();
@@ -37,7 +38,7 @@ public class DateTimeUtils {
             + "If not specified, Time fields will be assumed to be number of milliseconds since epoch (Midnight, Jan 1, 1970 GMT). "
             + "If specified, the value must match the Java Simple Date Format (for example, HH:mm:ss for a two-digit hour in 24-hour format, followed by "
             + "a two-digit minute, followed by a two-digit second, all separated by ':' characters, as in 18:04:15).")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .addValidator(new SimpleDateFormatValidator())
         .required(false)
         .build();
@@ -49,7 +50,7 @@ public class DateTimeUtils {
             + "If specified, the value must match the Java Simple Date Format (for example, MM/dd/yyyy HH:mm:ss for a two-digit month, followed by "
             + "a two-digit day, followed by a four-digit year, all separated by '/' characters; and then followed by a two-digit hour in 24-hour format, followed by "
             + "a two-digit minute, followed by a two-digit second, all separated by ':' characters, as in 01/01/2017 18:04:15).")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .addValidator(new SimpleDateFormatValidator())
         .required(false)
         .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java
index 3e31d60..e14b1bf 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java
@@ -37,6 +37,7 @@ public class PropertyDescriptorDTO {
     private Boolean sensitive;
     private Boolean dynamic;
     private Boolean supportsEl;
+    private String scopeEl;
     private String identifiesControllerService;
     private BundleDTO identifiesControllerServiceBundle;
 
@@ -167,6 +168,20 @@ public class PropertyDescriptorDTO {
     }
 
     /**
+     * @return specifies the scope of expression language evaluation
+     */
+    @ApiModelProperty(
+            value = "Scope of the EL evaluation for the property."
+    )
+    public String getScopeEl() {
+        return scopeEl;
+    }
+
+    public void setScopeEl(String scopeEl) {
+        this.scopeEl = scopeEl;
+    }
+
+    /**
      * @return if this property identifies a controller service this returns the fully qualified type, null otherwise
      */
     @ApiModelProperty(

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java
index 9294906..48fa109 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java
@@ -18,11 +18,12 @@ package org.apache.nifi.documentation.html;
 
 import org.apache.nifi.annotation.behavior.DynamicProperties;
 import org.apache.nifi.annotation.behavior.DynamicProperty;
-import org.apache.nifi.annotation.behavior.Restriction;
-import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
 import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
 import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
 import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.DeprecationNotice;
 import org.apache.nifi.annotation.documentation.SeeAlso;
@@ -34,6 +35,7 @@ import org.apache.nifi.components.ConfigurableComponent;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.documentation.DocumentationWriter;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.util.StringUtils;
 import org.slf4j.Logger;
@@ -522,14 +524,36 @@ public class HtmlDocumentationWriter implements DocumentationWriter {
 
                 if (property.isExpressionLanguageSupported()) {
                     xmlStreamWriter.writeEmptyElement("br");
-                    writeSimpleElement(xmlStreamWriter, "strong", "Supports Expression Language: true");
+                    String text = "Supports Expression Language: true";
+                    final String perFF = " (will be evaluated using flow file attributes and registry)";
+                    final String registry = " (will be evaluated using registry only)";
+                    final InputRequirement inputRequirement = configurableComponent.getClass().getAnnotation(InputRequirement.class);
+
+                    switch(property.getExpressionLanguageScope()) {
+                        case FLOWFILE_ATTRIBUTES:
+                            if(inputRequirement != null && inputRequirement.value().equals(Requirement.INPUT_FORBIDDEN)) {
+                                text += registry;
+                            } else {
+                                text += perFF;
+                            }
+                            break;
+                        case VARIABLE_REGISTRY:
+                            text += registry;
+                            break;
+                        case NONE:
+                        default:
+                            // in case legacy/deprecated method has been used to specify EL support
+                            text += " (undefined scope)";
+                            break;
+                    }
+
+                    writeSimpleElement(xmlStreamWriter, "strong", text);
                 }
                 xmlStreamWriter.writeEndElement();
 
                 xmlStreamWriter.writeEndElement();
             }
 
-            // TODO support dynamic properties...
             xmlStreamWriter.writeEndElement();
 
         } else {
@@ -555,7 +579,7 @@ public class HtmlDocumentationWriter implements DocumentationWriter {
     /**
      * Indicates whether or not the component contains at least one property that supports Expression Language.
      *
-     * @param component the component to interogate
+     * @param component the component to interrogate
      * @return whether or not the component contains at least one sensitive property.
      */
     private boolean containsExpressionLanguage(final ConfigurableComponent component) {
@@ -590,10 +614,32 @@ public class HtmlDocumentationWriter implements DocumentationWriter {
                 writeSimpleElement(xmlStreamWriter, "td", dynamicProperty.value(), false, "value");
                 xmlStreamWriter.writeStartElement("td");
                 xmlStreamWriter.writeCharacters(dynamicProperty.description());
-                if (dynamicProperty.supportsExpressionLanguage()) {
-                    xmlStreamWriter.writeEmptyElement("br");
-                    writeSimpleElement(xmlStreamWriter, "strong", "Supports Expression Language: true");
+
+                xmlStreamWriter.writeEmptyElement("br");
+                String text;
+
+                if(dynamicProperty.expressionLanguageScope().equals(ExpressionLanguageScope.NONE)) {
+                    if(dynamicProperty.supportsExpressionLanguage()) {
+                        text = "Supports Expression Language: true (undefined scope)";
+                    } else {
+                        text = "Supports Expression Language: false";
+                    }
+                } else {
+                    switch(dynamicProperty.expressionLanguageScope()) {
+                        case FLOWFILE_ATTRIBUTES:
+                            text = "Supports Expression Language: true (will be evaluated using flow file attributes and registry)";
+                            break;
+                        case VARIABLE_REGISTRY:
+                            text = "Supports Expression Language: true (will be evaluated using registry only)";
+                            break;
+                        case NONE:
+                        default:
+                            text = "Supports Expression Language: false";
+                            break;
+                    }
                 }
+
+                writeSimpleElement(xmlStreamWriter, "strong", text);
                 xmlStreamWriter.writeEndElement();
                 xmlStreamWriter.writeEndElement();
             }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/DeprecatedProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/DeprecatedProcessor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/DeprecatedProcessor.java
index 0c4ca95..dcd3efe 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/DeprecatedProcessor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/DeprecatedProcessor.java
@@ -16,6 +16,12 @@
  */
 package org.apache.nifi.documentation.example;
 
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
 import org.apache.nifi.annotation.behavior.DynamicProperty;
 import org.apache.nifi.annotation.behavior.DynamicRelationship;
 import org.apache.nifi.annotation.behavior.ReadsAttribute;
@@ -32,6 +38,7 @@ import org.apache.nifi.annotation.lifecycle.OnShutdown;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -40,12 +47,6 @@ import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
 @Tags({"one", "two", "three"})
 @CapabilityDescription("This is a processor that is used to test documentation.")
 @WritesAttributes({
@@ -53,46 +54,69 @@ import java.util.Set;
     @WritesAttribute(attribute = "second")})
 @ReadsAttribute(attribute = "incoming", description = "this specifies the format of the thing")
 @SeeAlso(value = {FullyDocumentedControllerService.class, FullyDocumentedReportingTask.class}, classNames = {"org.apache.nifi.processor.ExampleProcessor"})
-@DynamicProperty(name = "Relationship Name", supportsExpressionLanguage = true, value = "some XPath", description = "Routes FlowFiles to relationships based on XPath")
+@DynamicProperty(name = "Relationship Name", expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
+                    value = "some XPath", description = "Routes FlowFiles to relationships based on XPath")
 @DynamicRelationship(name = "name from dynamic property", description = "all files that match the properties XPath")
 @Stateful(scopes = {Scope.CLUSTER, Scope.LOCAL}, description = "state management description")
 @Restricted("processor restriction description")
 @DeprecationNotice(alternatives = {FullyDocumentedProcessor.class, FullyDocumentedReportingTask.class})
 public class DeprecatedProcessor extends AbstractProcessor {
 
-    public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder().name("Input Directory")
-            .description("The input directory from which to pull files").required(true)
+    public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder()
+            .name("Input Directory")
+            .description("The input directory from which to pull files")
+            .required(true)
             .addValidator(StandardValidators.createDirectoryExistsValidator(true, false))
-            .expressionLanguageSupported(true).build();
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
 
-    public static final PropertyDescriptor RECURSE = new PropertyDescriptor.Builder().name("Recurse Subdirectories")
-            .description("Indicates whether or not to pull files from subdirectories").required(true)
+    public static final PropertyDescriptor RECURSE = new PropertyDescriptor.Builder()
+            .name("Recurse Subdirectories")
+            .description("Indicates whether or not to pull files from subdirectories")
+            .required(true)
             .allowableValues(
                     new AllowableValue("true", "true", "Should pull from sub directories"),
                     new AllowableValue("false", "false", "Should not pull from sub directories")
-            ).defaultValue("true").build();
-
-    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor.Builder().name("Polling Interval")
-            .description("Indicates how long to wait before performing a directory listing").required(true)
-            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR).defaultValue("0 sec").build();
+            )
+            .defaultValue("true")
+            .build();
+
+    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor.Builder()
+            .name("Polling Interval")
+            .description("Indicates how long to wait before performing a directory listing")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("0 sec")
+            .build();
 
     public static final PropertyDescriptor OPTIONAL_PROPERTY = new PropertyDescriptor.Builder()
-            .name("Optional Property").description("This is a property you can use or not").required(false).build();
+            .name("Optional Property")
+            .description("This is a property you can use or not")
+            .required(false)
+            .build();
 
     public static final PropertyDescriptor TYPE_PROPERTY = new PropertyDescriptor.Builder()
             .name("Type")
             .description("This is the type of something that you can choose.  It has several possible values")
             .allowableValues("yes", "no", "maybe", "possibly", "not likely", "longer option name")
-            .required(true).build();
+            .required(true)
+            .build();
 
     public static final PropertyDescriptor SERVICE_PROPERTY = new PropertyDescriptor.Builder()
-            .name("Controller Service").description("This is the controller service to use to do things")
-            .identifiesControllerService(SampleService.class).required(true).build();
-
-    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
-            .description("Successful files").build();
-    public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure")
-            .description("Failing files").build();
+            .name("Controller Service")
+            .description("This is the controller service to use to do things")
+            .identifiesControllerService(SampleService.class)
+            .required(true)
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Successful files")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Failing files")
+            .build();
 
     private List<PropertyDescriptor> properties;
     private Set<Relationship> relationships;

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java
index 8442a33..efd8e99 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java
@@ -37,6 +37,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.RequiredPermission;
 import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -58,7 +59,8 @@ import java.util.Set;
     @WritesAttribute(attribute = "second")})
 @ReadsAttribute(attribute = "incoming", description = "this specifies the format of the thing")
 @SeeAlso(value = {FullyDocumentedControllerService.class, FullyDocumentedReportingTask.class}, classNames = {"org.apache.nifi.processor.ExampleProcessor"})
-@DynamicProperty(name = "Relationship Name", supportsExpressionLanguage = true, value = "some XPath", description = "Routes FlowFiles to relationships based on XPath")
+@DynamicProperty(name = "Relationship Name", expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
+                    value = "some XPath", description = "Routes FlowFiles to relationships based on XPath")
 @DynamicRelationship(name = "name from dynamic property", description = "all files that match the properties XPath")
 @Stateful(scopes = {Scope.CLUSTER, Scope.LOCAL}, description = "state management description")
 @Restricted(
@@ -75,39 +77,63 @@ import java.util.Set;
 @SystemResourceConsideration(resource = SystemResource.MEMORY, description = "")
 public class FullyDocumentedProcessor extends AbstractProcessor {
 
-    public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder().name("Input Directory")
-            .description("The input directory from which to pull files").required(true)
+    public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder()
+            .name("Input Directory")
+            .description("The input directory from which to pull files")
+            .required(true)
             .addValidator(StandardValidators.createDirectoryExistsValidator(true, false))
-            .expressionLanguageSupported(true).build();
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
 
-    public static final PropertyDescriptor RECURSE = new PropertyDescriptor.Builder().name("Recurse Subdirectories")
-            .description("Indicates whether or not to pull files from subdirectories").required(true)
+    public static final PropertyDescriptor RECURSE = new PropertyDescriptor.Builder()
+            .name("Recurse Subdirectories")
+            .description("Indicates whether or not to pull files from subdirectories")
+            .required(true)
             .allowableValues(
                     new AllowableValue("true", "true", "Should pull from sub directories"),
                     new AllowableValue("false", "false", "Should not pull from sub directories")
-            ).defaultValue("true").build();
-
-    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor.Builder().name("Polling Interval")
-            .description("Indicates how long to wait before performing a directory listing").required(true)
-            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR).defaultValue("0 sec").build();
+            )
+            .defaultValue("true")
+            .build();
+
+    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor.Builder()
+            .name("Polling Interval")
+            .description("Indicates how long to wait before performing a directory listing")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("0 sec")
+            .build();
 
     public static final PropertyDescriptor OPTIONAL_PROPERTY = new PropertyDescriptor.Builder()
-            .name("Optional Property").description("This is a property you can use or not").required(false).build();
+            .name("Optional Property")
+            .description("This is a property you can use or not")
+            .required(false)
+            .expressionLanguageSupported(true) // test documentation of deprecated method
+            .build();
 
     public static final PropertyDescriptor TYPE_PROPERTY = new PropertyDescriptor.Builder()
             .name("Type")
             .description("This is the type of something that you can choose.  It has several possible values")
             .allowableValues("yes", "no", "maybe", "possibly", "not likely", "longer option name")
-            .required(true).build();
+            .required(true)
+            .expressionLanguageSupported(false) // test documentation of deprecated method
+            .build();
 
     public static final PropertyDescriptor SERVICE_PROPERTY = new PropertyDescriptor.Builder()
-            .name("Controller Service").description("This is the controller service to use to do things")
-            .identifiesControllerService(SampleService.class).required(true).build();
-
-    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
-            .description("Successful files").build();
-    public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure")
-            .description("Failing files").build();
+            .name("Controller Service")
+            .description("This is the controller service to use to do things")
+            .identifiesControllerService(SampleService.class)
+            .required(true)
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Successful files")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Failing files")
+            .build();
 
     private List<PropertyDescriptor> properties;
     private Set<Relationship> relationships;

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java
index 10333d0..a047fd5 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java
@@ -16,8 +16,14 @@
  */
 package org.apache.nifi.documentation.html;
 
-import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import static org.apache.nifi.documentation.html.XmlValidator.assertContains;
+import static org.apache.nifi.documentation.html.XmlValidator.assertNotContains;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
 import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.components.RequiredPermission;
 import org.apache.nifi.documentation.DocumentationWriter;
@@ -29,12 +35,6 @@ import org.apache.nifi.init.ProcessorInitializer;
 import org.junit.Assert;
 import org.junit.Test;
 
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-
-import static org.apache.nifi.documentation.html.XmlValidator.assertContains;
-import static org.apache.nifi.documentation.html.XmlValidator.assertNotContains;
-
 public class ProcessorDocumentationWriterTest {
 
     @Test
@@ -86,6 +86,13 @@ public class ProcessorDocumentationWriterTest {
         assertNotContains(results, "No tags provided.");
         assertNotContains(results, "Additional Details...");
 
+        // check expression language scope
+        assertContains(results, "Supports Expression Language: true (will be evaluated using registry only)");
+        assertContains(results, "Supports Expression Language: true (undefined scope)");
+
+        // verify dynamic properties
+        assertContains(results, "Routes FlowFiles to relationships based on XPath");
+
         // input requirement
         assertContains(results, "This component does not allow an incoming relationship.");
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateUtils.java
index 6bfa49d..5b40f72 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateUtils.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateUtils.java
@@ -209,6 +209,7 @@ public class TemplateUtils {
         descriptor.setRequired(null);
         descriptor.setSensitive(null);
         descriptor.setSupportsEl(null);
+        descriptor.setScopeEl(null);
         descriptor.setIdentifiesControllerServiceBundle(null);
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/DummyScheduledReportingTask.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/DummyScheduledReportingTask.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/DummyScheduledReportingTask.java
index a347e98..4acd624 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/DummyScheduledReportingTask.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/DummyScheduledReportingTask.java
@@ -22,6 +22,7 @@ import java.util.List;
 
 import org.apache.nifi.annotation.configuration.DefaultSchedule;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.reporting.AbstractReportingTask;
 import org.apache.nifi.reporting.ReportingContext;
@@ -37,7 +38,7 @@ public class DummyScheduledReportingTask extends AbstractReportingTask {
         .name("Test with default value")
         .description("Test with default value")
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .defaultValue("nifi")
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .build();
@@ -46,7 +47,7 @@ public class DummyScheduledReportingTask extends AbstractReportingTask {
         .name("Test without default value")
         .description("Test without default value")
         .required(false)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .build();
 


[11/15] nifi git commit: NIFI-4149 - Indicate if EL is evaluated against FFs or not - take into account input requirement for documentation rendering - Renamed variable registry scope and added comments - Doc + change in mock framework to check scope + u

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchProcessor.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchProcessor.java
index 0453785..feac3c4 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchProcessor.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchProcessor.java
@@ -19,6 +19,7 @@ package org.apache.nifi.processors.elasticsearch;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.exception.ProcessException;
@@ -49,7 +50,7 @@ public abstract class AbstractElasticsearchProcessor extends AbstractProcessor {
             .required(true)
             .defaultValue("UTF-8")
             .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
@@ -57,7 +58,7 @@ public abstract class AbstractElasticsearchProcessor extends AbstractProcessor {
             .description("Username to access the Elasticsearch cluster")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
@@ -66,7 +67,7 @@ public abstract class AbstractElasticsearchProcessor extends AbstractProcessor {
             .required(false)
             .sensitive(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     protected abstract void createElasticsearchClient(ProcessContext context) throws ProcessException;

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchTransportClientProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchTransportClientProcessor.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchTransportClientProcessor.java
index b260768..c6f42a9 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchTransportClientProcessor.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchTransportClientProcessor.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.processors.elasticsearch;
 
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.exception.ProcessException;
@@ -49,7 +50,7 @@ public abstract class AbstractElasticsearchTransportClientProcessor extends Abst
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .defaultValue("elasticsearch")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     protected static final PropertyDescriptor HOSTS = new PropertyDescriptor.Builder()
@@ -58,9 +59,8 @@ public abstract class AbstractElasticsearchTransportClientProcessor extends Abst
                     + "host1:port,host2:port,....  For example testcluster:9300. This processor uses the Transport Client to "
                     + "connect to hosts. The default transport client port is 9300.")
             .required(true)
-            .expressionLanguageSupported(false)
             .addValidator(StandardValidators.HOSTNAME_PORT_LIST_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor PROP_SHIELD_LOCATION = new PropertyDescriptor.Builder()
@@ -71,7 +71,7 @@ public abstract class AbstractElasticsearchTransportClientProcessor extends Abst
                     + "lib/ directory, doing so will prevent the Shield plugin from being loaded.")
             .required(false)
             .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     protected static final PropertyDescriptor PING_TIMEOUT = new PropertyDescriptor.Builder()
@@ -81,7 +81,7 @@ public abstract class AbstractElasticsearchTransportClientProcessor extends Abst
             .required(true)
             .defaultValue("5s")
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     protected static final PropertyDescriptor SAMPLER_INTERVAL = new PropertyDescriptor.Builder()
@@ -91,7 +91,7 @@ public abstract class AbstractElasticsearchTransportClientProcessor extends Abst
             .required(true)
             .defaultValue("5s")
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     protected AtomicReference<Client> esClient = new AtomicReference<>();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/FetchElasticsearch.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/FetchElasticsearch.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/FetchElasticsearch.java
index 9edd3f9..9d7a402 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/FetchElasticsearch.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/FetchElasticsearch.java
@@ -26,6 +26,7 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.ProcessContext;
@@ -85,7 +86,7 @@ public class FetchElasticsearch extends AbstractElasticsearchTransportClientProc
             .name("Document Identifier")
             .description("The identifier for the document to be fetched")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -93,7 +94,7 @@ public class FetchElasticsearch extends AbstractElasticsearchTransportClientProc
             .name("Index")
             .description("The name of the index to read from")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -101,7 +102,7 @@ public class FetchElasticsearch extends AbstractElasticsearchTransportClientProc
             .name("Type")
             .description("The type of this document (used by Elasticsearch for indexing and searching)")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/FetchElasticsearchHttp.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/FetchElasticsearchHttp.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/FetchElasticsearchHttp.java
index 63163e4..2c692df 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/FetchElasticsearchHttp.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/FetchElasticsearchHttp.java
@@ -32,6 +32,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.ProcessContext;
@@ -70,7 +71,7 @@ import java.util.stream.Stream;
 @DynamicProperty(
         name = "A URL query parameter",
         value = "The value to set it to",
-        supportsExpressionLanguage = true,
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
         description = "Adds the specified property name/value as a query parameter in the Elasticsearch URL used for processing")
 public class FetchElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
 
@@ -102,7 +103,7 @@ public class FetchElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
             .displayName("Document Identifier")
             .description("The identifier of the document to be fetched")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -111,7 +112,7 @@ public class FetchElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
             .displayName("Index")
             .description("The name of the index to read from.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -121,7 +122,7 @@ public class FetchElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
             .description("The (optional) type of this document, used by Elasticsearch for indexing and searching. If the property is empty, "
                     + "the first document matching the identifier across all types will be retrieved.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -131,7 +132,7 @@ public class FetchElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
             .description("A comma-separated list of fields to retrieve from the document. If the Fields property is left blank, "
                     + "then the entire document's source will be retrieved.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearch.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearch.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearch.java
index 31af3db..f5082ea 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearch.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearch.java
@@ -28,6 +28,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.ProcessContext;
@@ -81,7 +82,7 @@ public class PutElasticsearch extends AbstractElasticsearchTransportClientProces
             .name("Identifier Attribute")
             .description("The name of the attribute containing the identifier for each FlowFile")
             .required(true)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .addValidator(StandardValidators.ATTRIBUTE_KEY_VALIDATOR)
             .build();
 
@@ -89,7 +90,7 @@ public class PutElasticsearch extends AbstractElasticsearchTransportClientProces
             .name("Index")
             .description("The name of the index to insert into")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(
                     AttributeExpression.ResultType.STRING, true))
             .build();
@@ -98,7 +99,7 @@ public class PutElasticsearch extends AbstractElasticsearchTransportClientProces
             .name("Type")
             .description("The type of this document (used by Elasticsearch for indexing and searching)")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
             .build();
 
@@ -106,7 +107,7 @@ public class PutElasticsearch extends AbstractElasticsearchTransportClientProces
             .name("Index Operation")
             .description("The type of the operation used to index (index, update, upsert)")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
             .defaultValue("index")
             .build();
@@ -117,7 +118,7 @@ public class PutElasticsearch extends AbstractElasticsearchTransportClientProces
             .required(true)
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
             .defaultValue("100")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     private static final Set<Relationship> relationships;

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearchHttp.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearchHttp.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearchHttp.java
index 9102bfb..caf1e01 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearchHttp.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearchHttp.java
@@ -38,6 +38,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.ProcessContext;
@@ -72,7 +73,7 @@ import static org.apache.commons.lang3.StringUtils.trimToEmpty;
 @DynamicProperty(
         name = "A URL query parameter",
         value = "The value to set it to",
-        supportsExpressionLanguage = true,
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
         description = "Adds the specified property name/value as a query parameter in the Elasticsearch URL used for processing")
 @SystemResourceConsideration(resource = SystemResource.MEMORY)
 public class PutElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
@@ -94,7 +95,7 @@ public class PutElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
                     + "this property may be left empty or evaluate to an empty value, in which case the document's identifier will be "
                     + "auto-generated by Elasticsearch. For all other Index Operations, the attribute must evaluate to a non-empty value.")
             .required(false)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .addValidator(StandardValidators.ATTRIBUTE_KEY_VALIDATOR)
             .build();
 
@@ -103,7 +104,7 @@ public class PutElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
             .displayName("Index")
             .description("The name of the index to insert into")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(
                     AttributeExpression.ResultType.STRING, true))
             .build();
@@ -113,7 +114,7 @@ public class PutElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
             .displayName("Type")
             .description("The type of this document (used by Elasticsearch for indexing and searching)")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
             .build();
 
@@ -122,7 +123,7 @@ public class PutElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
             .displayName("Index Operation")
             .description("The type of the operation used to index (index, update, upsert, delete)")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
             .defaultValue("index")
             .build();
@@ -136,7 +137,7 @@ public class PutElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
             .required(true)
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
             .defaultValue("100")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     private static final Set<Relationship> relationships;

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearchHttpRecord.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearchHttpRecord.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearchHttpRecord.java
index ae98d52..9ea22bc 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearchHttpRecord.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearchHttpRecord.java
@@ -37,6 +37,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.ProcessContext;
@@ -92,7 +93,7 @@ import static org.apache.commons.lang3.StringUtils.trimToEmpty;
 @DynamicProperty(
         name = "A URL query parameter",
         value = "The value to set it to",
-        supportsExpressionLanguage = true,
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
         description = "Adds the specified property name/value as a query parameter in the Elasticsearch URL used for processing")
 public class PutElasticsearchHttpRecord extends AbstractElasticsearchHttpProcessor {
 
@@ -122,7 +123,7 @@ public class PutElasticsearchHttpRecord extends AbstractElasticsearchHttpProcess
                     + "auto-generated by Elasticsearch. For all other Index Operations, the field's value must be non-empty.")
             .required(false)
             .addValidator(new RecordPathValidator())
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     static final PropertyDescriptor INDEX = new PropertyDescriptor.Builder()
@@ -130,7 +131,7 @@ public class PutElasticsearchHttpRecord extends AbstractElasticsearchHttpProcess
             .displayName("Index")
             .description("The name of the index to insert into")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(
                     AttributeExpression.ResultType.STRING, true))
             .build();
@@ -140,7 +141,7 @@ public class PutElasticsearchHttpRecord extends AbstractElasticsearchHttpProcess
             .displayName("Type")
             .description("The type of this document (used by Elasticsearch for indexing and searching)")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
             .build();
 
@@ -149,7 +150,7 @@ public class PutElasticsearchHttpRecord extends AbstractElasticsearchHttpProcess
             .displayName("Index Operation")
             .description("The type of the operation used to index (index, update, upsert, delete)")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
             .defaultValue("index")
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/QueryElasticsearchHttp.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/QueryElasticsearchHttp.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/QueryElasticsearchHttp.java
index 4b7312f..0f6ec46 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/QueryElasticsearchHttp.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/QueryElasticsearchHttp.java
@@ -32,6 +32,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.ProcessContext;
@@ -76,7 +77,7 @@ import java.util.stream.Stream;
 @DynamicProperty(
         name = "A URL query parameter",
         value = "The value to set it to",
-        supportsExpressionLanguage = true,
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
         description = "Adds the specified property name/value as a query parameter in the Elasticsearch URL used for processing")
 public class QueryElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
 
@@ -107,16 +108,22 @@ public class QueryElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
             .build();
 
     public static final PropertyDescriptor QUERY = new PropertyDescriptor.Builder()
-            .name("query-es-query").displayName("Query")
-            .description("The Lucene-style query to run against ElasticSearch (e.g., genre:blues AND -artist:muddy)").required(true)
-            .expressionLanguageSupported(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .name("query-es-query")
+            .displayName("Query")
+            .description("The Lucene-style query to run against ElasticSearch (e.g., genre:blues AND -artist:muddy)")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
     public static final PropertyDescriptor INDEX = new PropertyDescriptor.Builder()
-            .name("query-es-index").displayName("Index")
+            .name("query-es-index")
+            .displayName("Index")
             .description("The name of the index to read from. If the property is set "
-                            + "to _all, the query will match across all indexes.").required(true)
-            .expressionLanguageSupported(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                            + "to _all, the query will match across all indexes.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
     public static final PropertyDescriptor TYPE = new PropertyDescriptor.Builder()
@@ -126,7 +133,7 @@ public class QueryElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
                     "The (optional) type of this query, used by Elasticsearch for indexing and searching. If the property is empty, "
                             + "the the query will match across all types.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -137,7 +144,7 @@ public class QueryElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
                     "A comma-separated list of fields to retrieve from the document. If the Fields property is left blank, "
                             + "then the entire document's source will be retrieved.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -147,20 +154,29 @@ public class QueryElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
             .description(
                     "A sort parameter (e.g., timestamp:asc). If the Sort property is left blank, "
                             + "then the results will be retrieved in document order.")
-            .required(false).expressionLanguageSupported(true)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
+            .required(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
 
     public static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
-            .name("query-es-size").displayName("Page Size").defaultValue("20")
+            .name("query-es-size")
+            .displayName("Page Size")
+            .defaultValue("20")
             .description("Determines how many documents to return per page during scrolling.")
-            .required(true).expressionLanguageSupported(true)
-            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR).build();
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .build();
 
     public static final PropertyDescriptor LIMIT = new PropertyDescriptor.Builder()
-            .name("query-es-limit").displayName("Limit")
+            .name("query-es-limit")
+            .displayName("Limit")
             .description("If set, limits the number of results that will be returned.")
-            .required(false).expressionLanguageSupported(true)
-            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR).build();
+            .required(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .build();
 
     public static final PropertyDescriptor TARGET = new PropertyDescriptor.Builder()
             .name("query-es-target")
@@ -170,10 +186,12 @@ public class QueryElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
                             + "response will be written as the content of the flow file.  In the case of 'Flow file attributes', "
                             + "the original flow file (if applicable) will be cloned for each result, and all return fields will be placed "
                             + "in a flow file attribute of the same name, but prefixed by 'es.result.'")
-            .required(true).expressionLanguageSupported(false)
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .defaultValue(TARGET_FLOW_FILE_CONTENT)
             .allowableValues(TARGET_FLOW_FILE_CONTENT, TARGET_FLOW_FILE_ATTRIBUTES)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
 
     private static final Set<Relationship> relationships;
     private static final List<PropertyDescriptor> propertyDescriptors;

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/ScrollElasticsearchHttp.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/ScrollElasticsearchHttp.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/ScrollElasticsearchHttp.java
index d49e71f..f6f351a 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/ScrollElasticsearchHttp.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/ScrollElasticsearchHttp.java
@@ -36,6 +36,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.components.state.StateManager;
 import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.ProcessContext;
@@ -76,7 +77,7 @@ import java.util.stream.Stream;
 @DynamicProperty(
         name = "A URL query parameter",
         value = "The value to set it to",
-        supportsExpressionLanguage = true,
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
         description = "Adds the specified property name/value as a query parameter in the Elasticsearch URL used for processing")
 @Stateful(description = "After each successful scroll page, the latest scroll_id is persisted in scrollId as input for the next scroll call.  "
         + "Once the entire query is complete, finishedQuery state will be set to true, and the processor will not execute unless this is cleared.", scopes = { Scope.LOCAL })
@@ -100,9 +101,12 @@ public class ScrollElasticsearchHttp extends AbstractElasticsearchHttpProcessor
                             + "flow files will be routed to failure.").build();
 
     public static final PropertyDescriptor QUERY = new PropertyDescriptor.Builder()
-            .name("scroll-es-query").displayName("Query")
-            .description("The Lucene-style query to run against ElasticSearch (e.g., genre:blues AND -artist:muddy)").required(true)
-            .expressionLanguageSupported(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .name("scroll-es-query")
+            .displayName("Query")
+            .description("The Lucene-style query to run against ElasticSearch (e.g., genre:blues AND -artist:muddy)")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
     public static final PropertyDescriptor SCROLL_DURATION = new PropertyDescriptor.Builder()
@@ -111,7 +115,7 @@ public class ScrollElasticsearchHttp extends AbstractElasticsearchHttpProcessor
             .description("The scroll duration is how long each search context is kept in memory.")
             .defaultValue("1m")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(
                     StandardValidators.createRegexMatchingValidator(Pattern.compile("[0-9]+(m|h)")))
             .build();
@@ -122,7 +126,7 @@ public class ScrollElasticsearchHttp extends AbstractElasticsearchHttpProcessor
             .description("The name of the index to read from. If the property is set "
                             + "to _all, the query will match across all indexes.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -134,7 +138,8 @@ public class ScrollElasticsearchHttp extends AbstractElasticsearchHttpProcessor
                     + "the the query will match across all types.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .required(false)
-            .expressionLanguageSupported(true).build();
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
 
     public static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
             .name("scroll-es-fields")
@@ -142,8 +147,10 @@ public class ScrollElasticsearchHttp extends AbstractElasticsearchHttpProcessor
             .description(
                     "A comma-separated list of fields to retrieve from the document. If the Fields property is left blank, "
                             + "then the entire document's source will be retrieved.")
-            .required(false).expressionLanguageSupported(true)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
+            .required(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
 
     public static final PropertyDescriptor SORT = new PropertyDescriptor.Builder()
             .name("scroll-es-sort")
@@ -151,14 +158,20 @@ public class ScrollElasticsearchHttp extends AbstractElasticsearchHttpProcessor
             .description(
                     "A sort parameter (e.g., timestamp:asc). If the Sort property is left blank, "
                             + "then the results will be retrieved in document order.")
-            .required(false).expressionLanguageSupported(true)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
+            .required(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
 
     public static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
-            .name("scroll-es-size").displayName("Page Size").defaultValue("20")
+            .name("scroll-es-size")
+            .displayName("Page Size")
+            .defaultValue("20")
             .description("Determines how many documents to return per page during scrolling.")
-            .required(true).expressionLanguageSupported(true)
-            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR).build();
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .build();
 
     private static final Set<Relationship> relationships;
     private static final List<PropertyDescriptor> propertyDescriptors;

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/ITQueryElasticsearchHttp.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/ITQueryElasticsearchHttp.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/ITQueryElasticsearchHttp.java
index 15f2707..f60e2f9 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/ITQueryElasticsearchHttp.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/ITQueryElasticsearchHttp.java
@@ -40,7 +40,6 @@ public class ITQueryElasticsearchHttp {
     @Test
     public void testFetchElasticsearchOnTrigger() throws IOException {
         runner = TestRunners.newTestRunner(QueryElasticsearchHttp.class); // all docs are found
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL,
                 "http://localhost.internal:9200");
 
@@ -67,7 +66,6 @@ public class ITQueryElasticsearchHttp {
     @Test
     public void testFetchElasticsearchOnTrigger_IncomingFile() throws IOException {
         runner = TestRunners.newTestRunner(QueryElasticsearchHttp.class); // all docs are found
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL,
                 "http://localhost.internal:9200");
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/ITScrollElasticsearchHttp.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/ITScrollElasticsearchHttp.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/ITScrollElasticsearchHttp.java
index aa2a1e0..57ee242 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/ITScrollElasticsearchHttp.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/ITScrollElasticsearchHttp.java
@@ -38,7 +38,6 @@ public class ITScrollElasticsearchHttp {
     @Test
     public void testFetchElasticsearchOnTrigger() throws IOException {
         runner = TestRunners.newTestRunner(ScrollElasticsearchHttp.class); // all docs are found
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL,
                 "http://ip-172-31-49-152.ec2.internal:9200");
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestFetchElasticsearch.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestFetchElasticsearch.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestFetchElasticsearch.java
index b36b296..961c181 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestFetchElasticsearch.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestFetchElasticsearch.java
@@ -16,6 +16,23 @@
  */
 package org.apache.nifi.processors.elasticsearch;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.MalformedURLException;
+import java.util.HashMap;
+import java.util.concurrent.ExecutionException;
+
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.provenance.ProvenanceEventType;
@@ -44,23 +61,6 @@ import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.MalformedURLException;
-import java.util.HashMap;
-import java.util.concurrent.ExecutionException;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.when;
-
 public class TestFetchElasticsearch {
 
     private InputStream docExample;
@@ -81,7 +81,6 @@ public class TestFetchElasticsearch {
     @Test
     public void testFetchElasticsearchOnTrigger() throws IOException {
         runner = TestRunners.newTestRunner(new FetchElasticsearchTestProcessor(true)); // all docs are found
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.CLUSTER_NAME, "elasticsearch");
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.HOSTS, "127.0.0.1:9300");
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.PING_TIMEOUT, "5s");
@@ -110,7 +109,6 @@ public class TestFetchElasticsearch {
     @Test
     public void testFetchElasticsearchOnTriggerEL() throws IOException {
         runner = TestRunners.newTestRunner(new FetchElasticsearchTestProcessor(true)); // all docs are found
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.CLUSTER_NAME, "${cluster.name}");
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.HOSTS, "${hosts}");
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.PING_TIMEOUT, "${ping.timeout}");
@@ -147,7 +145,6 @@ public class TestFetchElasticsearch {
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.SAMPLER_INTERVAL, "5s");
         runner.setProperty(FetchElasticsearch.INDEX, "doc");
         runner.setProperty(FetchElasticsearch.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(FetchElasticsearch.DOC_ID, "${doc_id}");
 
         runner.enqueue(docExample, new HashMap<String, String>() {{
@@ -171,7 +168,6 @@ public class TestFetchElasticsearch {
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.SAMPLER_INTERVAL, "5s");
         runner.setProperty(FetchElasticsearch.INDEX, "doc");
         runner.setProperty(FetchElasticsearch.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(FetchElasticsearch.DOC_ID, "${doc_id}");
 
         runner.assertNotValid();
@@ -187,7 +183,6 @@ public class TestFetchElasticsearch {
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.SAMPLER_INTERVAL, "5s");
         runner.setProperty(FetchElasticsearch.INDEX, "doc");
         runner.setProperty(FetchElasticsearch.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(FetchElasticsearch.DOC_ID, "${doc_id}");
 
         // No Node Available exception
@@ -272,7 +267,6 @@ public class TestFetchElasticsearch {
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.SAMPLER_INTERVAL, "5s");
         runner.setProperty(FetchElasticsearch.INDEX, "doc");
         runner.setProperty(FetchElasticsearch.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(FetchElasticsearch.DOC_ID, "${doc_id}");
 
         // Allow time for the controller service to fully initialize
@@ -380,7 +374,6 @@ public class TestFetchElasticsearch {
         System.out.println("Starting test " + new Object() {
         }.getClass().getEnclosingMethod().getName());
         final TestRunner runner = TestRunners.newTestRunner(new FetchElasticsearch());
-        runner.setValidateExpressionUsage(true);
 
         //Local Cluster - Mac pulled from brew
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.CLUSTER_NAME, "elasticsearch_brew");
@@ -411,7 +404,6 @@ public class TestFetchElasticsearch {
         System.out.println("Starting test " + new Object() {
         }.getClass().getEnclosingMethod().getName());
         final TestRunner runner = TestRunners.newTestRunner(new FetchElasticsearch());
-        runner.setValidateExpressionUsage(true);
 
         //Local Cluster - Mac pulled from brew
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.CLUSTER_NAME, "elasticsearch_brew");

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestFetchElasticsearchHttp.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestFetchElasticsearchHttp.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestFetchElasticsearchHttp.java
index de56b49..bef54d4 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestFetchElasticsearchHttp.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestFetchElasticsearchHttp.java
@@ -16,14 +16,18 @@
  */
 package org.apache.nifi.processors.elasticsearch;
 
-import okhttp3.Call;
-import okhttp3.MediaType;
-import okhttp3.OkHttpClient;
-import okhttp3.Protocol;
-import okhttp3.Request;
-import okhttp3.RequestBody;
-import okhttp3.Response;
-import okhttp3.ResponseBody;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.util.HashMap;
+
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.ssl.SSLContextService;
@@ -37,17 +41,14 @@ import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URL;
-import java.util.HashMap;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+import okhttp3.Call;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Protocol;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
 
 public class TestFetchElasticsearchHttp {
 
@@ -68,7 +69,6 @@ public class TestFetchElasticsearchHttp {
     @Test
     public void testFetchElasticsearchOnTriggerEL() throws IOException {
         runner = TestRunners.newTestRunner(new FetchElasticsearchHttpTestProcessor(true)); // all docs are found
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "${es.url}");
 
         runner.setProperty(FetchElasticsearchHttp.INDEX, "doc");
@@ -97,7 +97,6 @@ public class TestFetchElasticsearchHttp {
     @Test
     public void testFetchElasticsearchOnTrigger() throws IOException {
         runner = TestRunners.newTestRunner(new FetchElasticsearchHttpTestProcessor(true)); // all docs are found
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(FetchElasticsearchHttp.INDEX, "doc");
@@ -124,7 +123,6 @@ public class TestFetchElasticsearchHttp {
         final String DOC_ID = "28039652140";
         FetchElasticsearchHttpTestProcessor processor = new FetchElasticsearchHttpTestProcessor(true);
         runner = TestRunners.newTestRunner(processor); // all docs are found
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, ES_URL);
 
         runner.setProperty(FetchElasticsearchHttp.INDEX, "doc");
@@ -149,7 +147,6 @@ public class TestFetchElasticsearchHttp {
     @Test
     public void testFetchElasticsearchOnTriggerWithFields() throws IOException {
         runner = TestRunners.newTestRunner(new FetchElasticsearchHttpTestProcessor(true)); // all docs are found
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(FetchElasticsearchHttp.INDEX, "doc");
@@ -177,7 +174,6 @@ public class TestFetchElasticsearchHttp {
         runner = TestRunners.newTestRunner(new FetchElasticsearchHttpTestProcessor(false)); // simulate doc not found
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(FetchElasticsearchHttp.INDEX, "doc");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(FetchElasticsearchHttp.DOC_ID, "${doc_id}");
 
         runner.setIncomingConnection(true);
@@ -201,7 +197,6 @@ public class TestFetchElasticsearchHttp {
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(FetchElasticsearchHttp.INDEX, "doc");
         runner.setProperty(FetchElasticsearchHttp.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(FetchElasticsearchHttp.DOC_ID, "${doc_id}");
 
         runner.enqueue(docExample, new HashMap<String, String>() {{
@@ -224,7 +219,6 @@ public class TestFetchElasticsearchHttp {
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(FetchElasticsearchHttp.INDEX, "doc");
         runner.setProperty(FetchElasticsearchHttp.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(FetchElasticsearchHttp.DOC_ID, "${doc_id}");
 
         runner.enqueue(docExample, new HashMap<String, String>() {{
@@ -247,7 +241,6 @@ public class TestFetchElasticsearchHttp {
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(FetchElasticsearchHttp.INDEX, "doc");
         runner.setProperty(FetchElasticsearchHttp.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(FetchElasticsearchHttp.DOC_ID, "${doc_id}");
 
         runner.setIncomingConnection(false);
@@ -264,7 +257,6 @@ public class TestFetchElasticsearchHttp {
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "127.0.0.1:9200");
         runner.setProperty(FetchElasticsearchHttp.INDEX, "doc");
         runner.setProperty(FetchElasticsearchHttp.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(FetchElasticsearchHttp.DOC_ID, "${doc_id}");
 
         runner.assertNotValid();
@@ -281,7 +273,6 @@ public class TestFetchElasticsearchHttp {
         runner.setProperty(FetchElasticsearchHttp.PROP_SSL_CONTEXT_SERVICE, "ssl-context");
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(FetchElasticsearchHttp.INDEX, "doc");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(FetchElasticsearchHttp.DOC_ID, "${doc_id}");
 
         // Allow time for the controller service to fully initialize
@@ -299,7 +290,6 @@ public class TestFetchElasticsearchHttp {
         FetchElasticsearchHttpTestProcessor p = new FetchElasticsearchHttpTestProcessor(true); // all docs are found
         p.setExpectedUrl("http://127.0.0.1:9200/doc/status/28039652140?_source_include=id&myparam=myvalue");
         runner = TestRunners.newTestRunner(p);
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(FetchElasticsearchHttp.INDEX, "doc");
@@ -391,6 +381,7 @@ public class TestFetchElasticsearchHttp {
             return url;
         }
 
+        @Override
         protected OkHttpClient getClient() {
             return client;
         }
@@ -413,7 +404,6 @@ public class TestFetchElasticsearchHttp {
         System.out.println("Starting test " + new Object() {
         }.getClass().getEnclosingMethod().getName());
         final TestRunner runner = TestRunners.newTestRunner(new FetchElasticsearchHttp());
-        runner.setValidateExpressionUsage(true);
 
         //Local Cluster - Mac pulled from brew
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
@@ -437,7 +427,6 @@ public class TestFetchElasticsearchHttp {
         System.out.println("Starting test " + new Object() {
         }.getClass().getEnclosingMethod().getName());
         final TestRunner runner = TestRunners.newTestRunner(new FetchElasticsearchHttp());
-        runner.setValidateExpressionUsage(true);
 
         //Local Cluster - Mac pulled from brew
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestPutElasticsearch.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestPutElasticsearch.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestPutElasticsearch.java
index 306f513..3fe6764 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestPutElasticsearch.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestPutElasticsearch.java
@@ -16,6 +16,21 @@
  */
 package org.apache.nifi.processors.elasticsearch;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.HashMap;
+import java.util.concurrent.ExecutionException;
+
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.provenance.ProvenanceEventType;
@@ -45,22 +60,6 @@ import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.HashMap;
-import java.util.concurrent.ExecutionException;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.when;
-
 public class TestPutElasticsearch {
 
     private InputStream docExample;
@@ -80,7 +79,6 @@ public class TestPutElasticsearch {
     @Test
     public void testPutElasticSearchOnTrigger() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(false)); // no failures
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.CLUSTER_NAME, "elasticsearch");
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.HOSTS, "127.0.0.1:9300");
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.PING_TIMEOUT, "5s");
@@ -110,7 +108,6 @@ public class TestPutElasticsearch {
     @Test
     public void testPutElasticSearchOnTriggerEL() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(false)); // no failures
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.CLUSTER_NAME, "${cluster.name}");
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.HOSTS, "${hosts}");
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.PING_TIMEOUT, "${ping.timeout}");
@@ -143,7 +140,6 @@ public class TestPutElasticsearch {
     @Test
     public void testPutElasticSearchOnTriggerWithFailures() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(true)); // simulate failures
-        runner.setValidateExpressionUsage(false);
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.CLUSTER_NAME, "elasticsearch");
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.HOSTS, "127.0.0.1:9300");
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.PING_TIMEOUT, "5s");
@@ -178,7 +174,6 @@ public class TestPutElasticsearch {
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.SAMPLER_INTERVAL, "5s");
         runner.setProperty(PutElasticsearch.INDEX, "doc");
         runner.setProperty(PutElasticsearch.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(PutElasticsearch.ID_ATTRIBUTE, "doc_id");
 
         // No Node Available exception
@@ -235,7 +230,6 @@ public class TestPutElasticsearch {
     @Test
     public void testPutElasticsearchOnTriggerWithNoIdAttribute() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(true)); // simulate failures
-        runner.setValidateExpressionUsage(false);
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.CLUSTER_NAME, "elasticsearch");
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.HOSTS, "127.0.0.1:9300");
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.PING_TIMEOUT, "5s");
@@ -256,7 +250,6 @@ public class TestPutElasticsearch {
     @Test
     public void testPutElasticsearchOnTriggerWithIndexFromAttribute() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(false));
-        runner.setValidateExpressionUsage(false);
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.CLUSTER_NAME, "elasticsearch");
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.HOSTS, "127.0.0.1:9300");
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.PING_TIMEOUT, "5s");
@@ -293,7 +286,6 @@ public class TestPutElasticsearch {
     @Test
     public void testPutElasticSearchOnTriggerWithInvalidIndexOp() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(false)); // no failures
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.CLUSTER_NAME, "elasticsearch");
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.HOSTS, "127.0.0.1:9300");
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.PING_TIMEOUT, "5s");
@@ -425,7 +417,6 @@ public class TestPutElasticsearch {
         System.out.println("Starting test " + new Object() {
         }.getClass().getEnclosingMethod().getName());
         final TestRunner runner = TestRunners.newTestRunner(new PutElasticsearch());
-        runner.setValidateExpressionUsage(false);
 
         //Local Cluster - Mac pulled from brew
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.CLUSTER_NAME, "elasticsearch_brew");
@@ -457,7 +448,6 @@ public class TestPutElasticsearch {
         System.out.println("Starting test " + new Object() {
         }.getClass().getEnclosingMethod().getName());
         final TestRunner runner = TestRunners.newTestRunner(new PutElasticsearch());
-        runner.setValidateExpressionUsage(false);
 
         //Local Cluster - Mac pulled from brew
         runner.setProperty(AbstractElasticsearchTransportClientProcessor.CLUSTER_NAME, "elasticsearch_brew");

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestPutElasticsearchHttp.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestPutElasticsearchHttp.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestPutElasticsearchHttp.java
index 5ca3814..0d36edd 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestPutElasticsearchHttp.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestPutElasticsearchHttp.java
@@ -16,13 +16,17 @@
  */
 package org.apache.nifi.processors.elasticsearch;
 
-import okhttp3.Call;
-import okhttp3.MediaType;
-import okhttp3.OkHttpClient;
-import okhttp3.Protocol;
-import okhttp3.Request;
-import okhttp3.Response;
-import okhttp3.ResponseBody;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.net.ConnectException;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+
 import org.apache.commons.io.IOUtils;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.exception.ProcessException;
@@ -36,16 +40,13 @@ import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
-import java.io.IOException;
-import java.net.ConnectException;
-import java.nio.charset.StandardCharsets;
-import java.util.HashMap;
-
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+import okhttp3.Call;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Protocol;
+import okhttp3.Request;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
 
 public class TestPutElasticsearchHttp {
 
@@ -66,7 +67,6 @@ public class TestPutElasticsearchHttp {
     @Test
     public void testPutElasticSearchOnTriggerIndex() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(false)); // no failures
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
@@ -88,7 +88,6 @@ public class TestPutElasticsearchHttp {
     @Test
     public void testPutElasticSearchOnTriggerUpdate() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(false)); // no failures
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
@@ -110,7 +109,6 @@ public class TestPutElasticsearchHttp {
     @Test
     public void testPutElasticSearchOnTriggerDelete() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(false)); // no failures
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
@@ -132,7 +130,6 @@ public class TestPutElasticsearchHttp {
     @Test
     public void testPutElasticSearchOnTriggerEL() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(false)); // no failures
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "${es.url}");
 
         runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
@@ -159,7 +156,6 @@ public class TestPutElasticsearchHttp {
     @Test
     public void testPutElasticSearchOnTriggerBadIndexOp() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(false)); // no failures
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
@@ -181,7 +177,6 @@ public class TestPutElasticsearchHttp {
     @Test
     public void testPutElasticSearchInvalidConfig() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(false)); // no failures
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
@@ -202,7 +197,6 @@ public class TestPutElasticsearchHttp {
         PutElasticsearchTestProcessor processor = new PutElasticsearchTestProcessor(true);
         processor.setStatus(100, "Should fail");
         runner = TestRunners.newTestRunner(processor); // simulate failures
-        runner.setValidateExpressionUsage(false);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
         runner.setProperty(PutElasticsearchHttp.TYPE, "status");
@@ -229,7 +223,6 @@ public class TestPutElasticsearchHttp {
         PutElasticsearchTestProcessor processor = new PutElasticsearchTestProcessor(true);
         processor.setStatus(-1, "Connection Exception");
         runner = TestRunners.newTestRunner(processor); // simulate failures
-        runner.setValidateExpressionUsage(false);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
         runner.setProperty(PutElasticsearchHttp.TYPE, "status");
@@ -246,7 +239,6 @@ public class TestPutElasticsearchHttp {
     @Test
     public void testPutElasticsearchOnTriggerWithNoIdAttribute() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(true)); // simulate failures
-        runner.setValidateExpressionUsage(false);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
         runner.setProperty(PutElasticsearchHttp.TYPE, "status");
@@ -266,7 +258,6 @@ public class TestPutElasticsearchHttp {
     @Test
     public void testPutElasticsearchOnTriggerWithIndexFromAttribute() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(false));
-        runner.setValidateExpressionUsage(false);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(PutElasticsearchHttp.INDEX, "${i}");
         runner.setProperty(PutElasticsearchHttp.TYPE, "${type}");
@@ -300,7 +291,6 @@ public class TestPutElasticsearchHttp {
     @Test
     public void testPutElasticSearchOnTriggerWithInvalidIndexOp() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(false)); // no failures
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
@@ -330,7 +320,6 @@ public class TestPutElasticsearchHttp {
         p.setExpectedUrl("http://127.0.0.1:9200/_bulk?pipeline=my-pipeline");
 
         runner = TestRunners.newTestRunner(p);
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
         runner.setProperty(PutElasticsearchHttp.TYPE, "status");
@@ -355,7 +344,6 @@ public class TestPutElasticsearchHttp {
         PutElasticsearchTestProcessor processor = new PutElasticsearchTestProcessor(true);
         processor.setResultField("not_found");
         runner = TestRunners.newTestRunner(processor); // simulate failures
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(PutElasticsearchHttp.INDEX_OP, "delete");
         runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
@@ -447,6 +435,7 @@ public class TestPutElasticsearchHttp {
             });
         }
 
+        @Override
         protected OkHttpClient getClient() {
             return client;
         }
@@ -469,7 +458,6 @@ public class TestPutElasticsearchHttp {
         System.out.println("Starting test " + new Object() {
         }.getClass().getEnclosingMethod().getName());
         final TestRunner runner = TestRunners.newTestRunner(new PutElasticsearchHttp());
-        runner.setValidateExpressionUsage(false);
 
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
@@ -493,7 +481,6 @@ public class TestPutElasticsearchHttp {
         System.out.println("Starting test " + new Object() {
         }.getClass().getEnclosingMethod().getName());
         final TestRunner runner = TestRunners.newTestRunner(new PutElasticsearchHttp());
-        runner.setValidateExpressionUsage(false);
 
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(PutElasticsearchHttp.INDEX, "doc");

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestPutElasticsearchHttpRecord.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestPutElasticsearchHttpRecord.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestPutElasticsearchHttpRecord.java
index 75fb6ec..2816aa2 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestPutElasticsearchHttpRecord.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestPutElasticsearchHttpRecord.java
@@ -16,13 +16,18 @@
  */
 package org.apache.nifi.processors.elasticsearch;
 
-import okhttp3.Call;
-import okhttp3.MediaType;
-import okhttp3.OkHttpClient;
-import okhttp3.Protocol;
-import okhttp3.Request;
-import okhttp3.Response;
-import okhttp3.ResponseBody;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.net.ConnectException;
+import java.util.HashMap;
+import java.util.List;
+
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.provenance.ProvenanceEventRecord;
@@ -37,17 +42,13 @@ import org.junit.After;
 import org.junit.Ignore;
 import org.junit.Test;
 
-import java.io.IOException;
-import java.net.ConnectException;
-import java.util.HashMap;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+import okhttp3.Call;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Protocol;
+import okhttp3.Request;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
 
 public class TestPutElasticsearchHttpRecord {
 
@@ -62,7 +63,6 @@ public class TestPutElasticsearchHttpRecord {
     public void testPutElasticSearchOnTriggerIndex() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearchHttpRecordTestProcessor(false)); // no failures
         generateTestData();
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(PutElasticsearchHttpRecord.INDEX, "doc");
@@ -88,7 +88,6 @@ public class TestPutElasticsearchHttpRecord {
     public void testPutElasticSearchOnTriggerUpdate() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearchHttpRecordTestProcessor(false)); // no failures
         generateTestData();
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(PutElasticsearchHttpRecord.INDEX, "doc");
@@ -110,7 +109,6 @@ public class TestPutElasticsearchHttpRecord {
     public void testPutElasticSearchOnTriggerDelete() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearchHttpRecordTestProcessor(false)); // no failures
         generateTestData();
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(PutElasticsearchHttpRecord.INDEX, "doc");
@@ -132,7 +130,6 @@ public class TestPutElasticsearchHttpRecord {
     public void testPutElasticSearchOnTriggerEL() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearchHttpRecordTestProcessor(false)); // no failures
         generateTestData();
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "${es.url}");
 
         runner.setProperty(PutElasticsearchHttpRecord.INDEX, "doc");
@@ -159,7 +156,6 @@ public class TestPutElasticsearchHttpRecord {
     public void testPutElasticSearchOnTriggerBadIndexOp() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearchHttpRecordTestProcessor(false)); // no failures
         generateTestData();
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(PutElasticsearchHttpRecord.INDEX, "doc");
@@ -181,7 +177,6 @@ public class TestPutElasticsearchHttpRecord {
     public void testPutElasticSearchInvalidConfig() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearchHttpRecordTestProcessor(false)); // no failures
         generateTestData();
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(PutElasticsearchHttpRecord.INDEX, "doc");
@@ -276,7 +271,6 @@ public class TestPutElasticsearchHttpRecord {
     public void testPutElasticsearchOnTriggerWithIndexFromAttribute() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearchHttpRecordTestProcessor(false));
         generateTestData();
-        runner.setValidateExpressionUsage(false);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(PutElasticsearchHttpRecord.INDEX, "${i}");
         runner.setProperty(PutElasticsearchHttpRecord.TYPE, "${type}");
@@ -310,7 +304,6 @@ public class TestPutElasticsearchHttpRecord {
     public void testPutElasticSearchOnTriggerWithInvalidIndexOp() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearchHttpRecordTestProcessor(false)); // no failures
         generateTestData();
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(PutElasticsearchHttpRecord.INDEX, "doc");
@@ -339,7 +332,6 @@ public class TestPutElasticsearchHttpRecord {
         p.setExpectedUrl("http://127.0.0.1:9200/_bulk?pipeline=my-pipeline");
         runner = TestRunners.newTestRunner(p);
         generateTestData();
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
 
         runner.setProperty(PutElasticsearchHttpRecord.INDEX, "doc");
@@ -427,6 +419,7 @@ public class TestPutElasticsearchHttpRecord {
             });
         }
 
+        @Override
         protected OkHttpClient getClient() {
             return client;
         }
@@ -449,7 +442,6 @@ public class TestPutElasticsearchHttpRecord {
         System.out.println("Starting test " + new Object() {
         }.getClass().getEnclosingMethod().getName());
         final TestRunner runner = TestRunners.newTestRunner(new PutElasticsearchHttpRecord());
-        runner.setValidateExpressionUsage(false);
 
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(PutElasticsearchHttpRecord.INDEX, "doc");
@@ -476,7 +468,6 @@ public class TestPutElasticsearchHttpRecord {
         System.out.println("Starting test " + new Object() {
         }.getClass().getEnclosingMethod().getName());
         final TestRunner runner = TestRunners.newTestRunner(new PutElasticsearchHttpRecord());
-        runner.setValidateExpressionUsage(false);
 
         runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
         runner.setProperty(PutElasticsearchHttpRecord.INDEX, "doc");


[08/15] nifi git commit: NIFI-4149 - Indicate if EL is evaluated against FFs or not - take into account input requirement for documentation rendering - Renamed variable registry scope and added comments - Doc + change in mock framework to check scope + u

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/ExtractHL7Attributes.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/ExtractHL7Attributes.java b/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/ExtractHL7Attributes.java
index 4ceb12e..6039aa0 100644
--- a/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/ExtractHL7Attributes.java
+++ b/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/ExtractHL7Attributes.java
@@ -41,6 +41,7 @@ import org.apache.nifi.annotation.behavior.SupportsBatching;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -88,7 +89,7 @@ public class ExtractHL7Attributes extends AbstractProcessor {
             .displayName("Character Encoding")
             .description("The Character Encoding that is used to encode the HL7 data")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
             .defaultValue("UTF-8")
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/RouteHL7.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/RouteHL7.java b/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/RouteHL7.java
index ba90370..6a8df89 100644
--- a/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/RouteHL7.java
+++ b/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/RouteHL7.java
@@ -41,6 +41,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.hl7.hapi.HapiMessage;
 import org.apache.nifi.hl7.model.HL7Message;
@@ -81,7 +82,7 @@ public class RouteHL7 extends AbstractProcessor {
             .name("Character Encoding")
             .description("The Character Encoding that is used to encode the HL7 data")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
             .defaultValue("UTF-8")
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-html-bundle/nifi-html-processors/src/main/java/org/apache/nifi/AbstractHTMLProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-html-bundle/nifi-html-processors/src/main/java/org/apache/nifi/AbstractHTMLProcessor.java b/nifi-nar-bundles/nifi-html-bundle/nifi-html-processors/src/main/java/org/apache/nifi/AbstractHTMLProcessor.java
index 8ad6f8a..902f215 100644
--- a/nifi-nar-bundles/nifi-html-bundle/nifi-html-processors/src/main/java/org/apache/nifi/AbstractHTMLProcessor.java
+++ b/nifi-nar-bundles/nifi-html-bundle/nifi-html-processors/src/main/java/org/apache/nifi/AbstractHTMLProcessor.java
@@ -20,6 +20,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -68,7 +69,7 @@ public abstract class AbstractHTMLProcessor extends AbstractProcessor {
                     " when an attribute value is extracted from a HTML element.")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor CSS_SELECTOR = new PropertyDescriptor
@@ -76,7 +77,7 @@ public abstract class AbstractHTMLProcessor extends AbstractProcessor {
             .description("CSS selector syntax string used to extract the desired HTML element(s).")
             .required(true)
             .addValidator(CSS_SELECTOR_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor HTML_CHARSET = new PropertyDescriptor

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-html-bundle/nifi-html-processors/src/main/java/org/apache/nifi/GetHTMLElement.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-html-bundle/nifi-html-processors/src/main/java/org/apache/nifi/GetHTMLElement.java b/nifi-nar-bundles/nifi-html-bundle/nifi-html-processors/src/main/java/org/apache/nifi/GetHTMLElement.java
index 713fabd..8346f03 100644
--- a/nifi-nar-bundles/nifi-html-bundle/nifi-html-processors/src/main/java/org/apache/nifi/GetHTMLElement.java
+++ b/nifi-nar-bundles/nifi-html-bundle/nifi-html-processors/src/main/java/org/apache/nifi/GetHTMLElement.java
@@ -20,6 +20,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.SupportsBatching;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -77,7 +78,7 @@ public class GetHTMLElement
             .description("Prepends the specified value to the resulting Element")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor APPEND_ELEMENT_VALUE = new PropertyDescriptor
@@ -85,7 +86,7 @@ public class GetHTMLElement
             .description("Appends the specified value to the resulting Element")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor ATTRIBUTE_KEY = new PropertyDescriptor
@@ -97,10 +98,9 @@ public class GetHTMLElement
                     " an absolute URL form using the specified base URL."))
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
-
     public static final PropertyDescriptor OUTPUT_TYPE = new PropertyDescriptor.Builder()
             .name("Output Type")
             .description("Controls the type of DOM value that is retrieved from the HTML element.")

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-html-bundle/nifi-html-processors/src/main/java/org/apache/nifi/ModifyHTMLElement.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-html-bundle/nifi-html-processors/src/main/java/org/apache/nifi/ModifyHTMLElement.java b/nifi-nar-bundles/nifi-html-bundle/nifi-html-processors/src/main/java/org/apache/nifi/ModifyHTMLElement.java
index 7f6e12e..24f9741 100644
--- a/nifi-nar-bundles/nifi-html-bundle/nifi-html-processors/src/main/java/org/apache/nifi/ModifyHTMLElement.java
+++ b/nifi-nar-bundles/nifi-html-bundle/nifi-html-processors/src/main/java/org/apache/nifi/ModifyHTMLElement.java
@@ -24,6 +24,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -79,7 +80,7 @@ public class ModifyHTMLElement extends AbstractHTMLProcessor {
             .description("Value to update the found HTML elements with")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor ATTRIBUTE_KEY = new PropertyDescriptor
@@ -88,7 +89,7 @@ public class ModifyHTMLElement extends AbstractHTMLProcessor {
                     " which attribute on the selected element will be modified with the new value."))
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     private List<PropertyDescriptor> descriptors;

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-html-bundle/nifi-html-processors/src/main/java/org/apache/nifi/PutHTMLElement.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-html-bundle/nifi-html-processors/src/main/java/org/apache/nifi/PutHTMLElement.java b/nifi-nar-bundles/nifi-html-bundle/nifi-html-processors/src/main/java/org/apache/nifi/PutHTMLElement.java
index bc9b70c..0d112cb 100644
--- a/nifi-nar-bundles/nifi-html-bundle/nifi-html-processors/src/main/java/org/apache/nifi/PutHTMLElement.java
+++ b/nifi-nar-bundles/nifi-html-bundle/nifi-html-processors/src/main/java/org/apache/nifi/PutHTMLElement.java
@@ -22,6 +22,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -78,7 +79,7 @@ public class PutHTMLElement extends AbstractHTMLProcessor {
                     "encoded in the resulting output.")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     private List<PropertyDescriptor> descriptors;
@@ -133,7 +134,7 @@ public class PutHTMLElement extends AbstractHTMLProcessor {
         final Elements eles;
         try {
             doc = parseHTMLDocumentFromFlowfile(flowFile, context, session);
-            eles = doc.select(context.getProperty(CSS_SELECTOR).evaluateAttributeExpressions().getValue());
+            eles = doc.select(context.getProperty(CSS_SELECTOR).evaluateAttributeExpressions(flowFile).getValue());
         } catch (Exception ex) {
             getLogger().error("Failed to extract HTML from {} due to {}; routing to {}", new Object[] {flowFile, ex.toString(), REL_INVALID_HTML.getName()}, ex);
             session.transfer(flowFile, REL_INVALID_HTML);

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-ignite-bundle/nifi-ignite-processors/src/main/java/org/apache/nifi/processors/ignite/cache/AbstractIgniteCacheProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-ignite-bundle/nifi-ignite-processors/src/main/java/org/apache/nifi/processors/ignite/cache/AbstractIgniteCacheProcessor.java b/nifi-nar-bundles/nifi-ignite-bundle/nifi-ignite-processors/src/main/java/org/apache/nifi/processors/ignite/cache/AbstractIgniteCacheProcessor.java
index ca6136c..2ea7343 100644
--- a/nifi-nar-bundles/nifi-ignite-bundle/nifi-ignite-processors/src/main/java/org/apache/nifi/processors/ignite/cache/AbstractIgniteCacheProcessor.java
+++ b/nifi-nar-bundles/nifi-ignite-bundle/nifi-ignite-processors/src/main/java/org/apache/nifi/processors/ignite/cache/AbstractIgniteCacheProcessor.java
@@ -23,6 +23,7 @@ import java.util.Set;
 import org.apache.ignite.IgniteCache;
 import org.apache.nifi.annotation.lifecycle.OnShutdown;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.Relationship;
@@ -56,7 +57,7 @@ public abstract class AbstractIgniteCacheProcessor extends AbstractIgniteProcess
                 "for determining Ignite cache key for the Flow File content")
             .required(true)
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(ResultType.STRING, true))
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     /**

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-influxdb-bundle/nifi-influxdb-processors/src/main/java/org/apache/nifi/processors/influxdb/AbstractInfluxDBProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-influxdb-bundle/nifi-influxdb-processors/src/main/java/org/apache/nifi/processors/influxdb/AbstractInfluxDBProcessor.java b/nifi-nar-bundles/nifi-influxdb-bundle/nifi-influxdb-processors/src/main/java/org/apache/nifi/processors/influxdb/AbstractInfluxDBProcessor.java
index 13838de..cedde21 100644
--- a/nifi-nar-bundles/nifi-influxdb-bundle/nifi-influxdb-processors/src/main/java/org/apache/nifi/processors/influxdb/AbstractInfluxDBProcessor.java
+++ b/nifi-nar-bundles/nifi-influxdb-bundle/nifi-influxdb-processors/src/main/java/org/apache/nifi/processors/influxdb/AbstractInfluxDBProcessor.java
@@ -23,6 +23,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.ProcessContext;
@@ -44,7 +45,7 @@ abstract class AbstractInfluxDBProcessor extends AbstractProcessor {
             .description("Specifies the character set of the document data.")
             .required(true)
             .defaultValue("UTF-8")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
             .build();
 
@@ -54,7 +55,7 @@ abstract class AbstractInfluxDBProcessor extends AbstractProcessor {
             .description("InfluxDB URL to connect to. Eg: http://influxdb:8086")
             .defaultValue("http://localhost:8086")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.URL_VALIDATOR)
             .build();
 
@@ -72,7 +73,7 @@ abstract class AbstractInfluxDBProcessor extends AbstractProcessor {
             .displayName("Database Name")
             .description("InfluxDB database to connect to")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -81,7 +82,7 @@ abstract class AbstractInfluxDBProcessor extends AbstractProcessor {
             .displayName("Username")
             .required(false)
             .description("Username for accessing InfluxDB")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -90,7 +91,7 @@ abstract class AbstractInfluxDBProcessor extends AbstractProcessor {
             .displayName("Password")
             .required(false)
             .description("Password for user")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .sensitive(true)
             .build();
@@ -99,7 +100,7 @@ abstract class AbstractInfluxDBProcessor extends AbstractProcessor {
             .name("influxdb-max-records-size")
             .displayName("Max size of records")
             .description("Maximum size of records allowed to be posted in one batch")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue("1 MB")
             .required(true)
             .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-influxdb-bundle/nifi-influxdb-processors/src/main/java/org/apache/nifi/processors/influxdb/PutInfluxDB.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-influxdb-bundle/nifi-influxdb-processors/src/main/java/org/apache/nifi/processors/influxdb/PutInfluxDB.java b/nifi-nar-bundles/nifi-influxdb-bundle/nifi-influxdb-processors/src/main/java/org/apache/nifi/processors/influxdb/PutInfluxDB.java
index ed45025..f507768 100644
--- a/nifi-nar-bundles/nifi-influxdb-bundle/nifi-influxdb-processors/src/main/java/org/apache/nifi/processors/influxdb/PutInfluxDB.java
+++ b/nifi-nar-bundles/nifi-influxdb-bundle/nifi-influxdb-processors/src/main/java/org/apache/nifi/processors/influxdb/PutInfluxDB.java
@@ -27,6 +27,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -67,7 +68,7 @@ public class PutInfluxDB extends AbstractInfluxDBProcessor {
             .description("InfluxDB consistency level")
             .required(true)
             .defaultValue(CONSISTENCY_LEVEL_ONE.getValue())
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .allowableValues(CONSISTENCY_LEVEL_ONE, CONSISTENCY_LEVEL_ANY, CONSISTENCY_LEVEL_ALL, CONSISTENCY_LEVEL_QUORUM)
             .build();
 
@@ -77,7 +78,7 @@ public class PutInfluxDB extends AbstractInfluxDBProcessor {
             .description("Retention policy for the saving the records")
             .defaultValue("autogen")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProvider.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProvider.java
index 5c822be..e9c24d5 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProvider.java
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProvider.java
@@ -39,6 +39,7 @@ import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.ssl.SSLContextService;
@@ -88,7 +89,7 @@ public class JMSConnectionFactoryProvider extends AbstractControllerService impl
                     + "class (i.e., org.apache.activemq.ActiveMQConnectionFactory)")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
     public static final PropertyDescriptor CLIENT_LIB_DIR_PATH = new PropertyDescriptor.Builder()
             .name(CF_LIB)
@@ -98,7 +99,7 @@ public class JMSConnectionFactoryProvider extends AbstractControllerService impl
                     + "ConnectionFactory implementation.")
             .addValidator(new ClientLibValidator())
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     // ConnectionFactory specific properties
@@ -109,7 +110,7 @@ public class JMSConnectionFactoryProvider extends AbstractControllerService impl
                     + "'tcp://myhost:61616' for ActiveMQ or 'myhost:1414' for IBM MQ")
             .addValidator(new NonEmptyBrokerURIValidator())
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
@@ -133,7 +134,9 @@ public class JMSConnectionFactoryProvider extends AbstractControllerService impl
         return new PropertyDescriptor.Builder()
                 .description("Specifies the value for '" + propertyDescriptorName
                         + "' property to be set on the provided ConnectionFactory implementation.")
-                .name(propertyDescriptorName).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).dynamic(true)
+                .name(propertyDescriptorName)
+                .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                .dynamic(true)
                 .build();
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
index 1ac468c..1b8a8f3 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
@@ -28,6 +28,7 @@ import javax.jms.ConnectionFactory;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.jms.cf.JMSConnectionFactoryProvider;
 import org.apache.nifi.jms.cf.JMSConnectionFactoryProviderDefinition;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -73,7 +74,7 @@ abstract class AbstractJMSProcessor<T extends JMSWorker> extends AbstractProcess
             .description("The name of the JMS Destination. Usually provided by the administrator (e.g., 'topic://myTopic' or 'myTopic').")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     static final PropertyDescriptor DESTINATION_TYPE = new PropertyDescriptor.Builder()
             .name("Destination Type")
@@ -90,7 +91,7 @@ abstract class AbstractJMSProcessor<T extends JMSWorker> extends AbstractProcess
                          "Please see JMS spec for further details")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
     static final PropertyDescriptor SESSION_CACHE_SIZE = new PropertyDescriptor.Builder()
             .name("Session Cache size")
@@ -114,7 +115,7 @@ abstract class AbstractJMSProcessor<T extends JMSWorker> extends AbstractProcess
             .required(true)
             .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
             .defaultValue(Charset.defaultCharset().name())
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
index e3e9a75..b7103ff 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
@@ -36,6 +36,7 @@ import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.jms.cf.JMSConnectionFactoryProvider;
 import org.apache.nifi.jms.processors.JMSConsumer.ConsumerCallback;
@@ -106,7 +107,7 @@ public class ConsumeJMS extends AbstractJMSProcessor<JMSConsumer> {
             .description("If destination is Topic if present then make it the consumer durable. " +
                          "@see https://docs.oracle.com/javaee/7/api/javax/jms/Session.html#createDurableConsumer-javax.jms.Topic-java.lang.String-")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue("false")
             .allowableValues("true", "false")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
@@ -116,7 +117,7 @@ public class ConsumeJMS extends AbstractJMSProcessor<JMSConsumer> {
             .description("If destination is Topic if present then make it the consumer shared. " +
                          "@see https://docs.oracle.com/javaee/7/api/javax/jms/Session.html#createSharedConsumer-javax.jms.Topic-java.lang.String-")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue("false")
             .allowableValues("true", "false")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
@@ -126,7 +127,7 @@ public class ConsumeJMS extends AbstractJMSProcessor<JMSConsumer> {
             .description("The name of the subscription to use if destination is Topic and is shared or durable.")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final Relationship REL_SUCCESS = new Relationship.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/PublishJMS.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/PublishJMS.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/PublishJMS.java
index c555fd7..f58b9cf 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/PublishJMS.java
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/PublishJMS.java
@@ -128,7 +128,7 @@ public class PublishJMS extends AbstractJMSProcessor<JMSPublisher> {
                         break;
                 }
                 processSession.transfer(flowFile, REL_SUCCESS);
-                processSession.getProvenanceReporter().send(flowFile, context.getProperty(DESTINATION).evaluateAttributeExpressions().getValue());
+                processSession.getProvenanceReporter().send(flowFile, destinationName);
             } catch (Exception e) {
                 processSession.transfer(flowFile, REL_FAILURE);
                 this.getLogger().error("Failed while sending message to JMS via " + publisher, e);

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_0_10.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_0_10.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_0_10.java
index 3c0bddc..f0026a4 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_0_10.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_0_10.java
@@ -44,6 +44,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -87,7 +88,7 @@ public class ConsumeKafkaRecord_0_10 extends AbstractProcessor {
             .description("The name of the Kafka Topic(s) to pull from. More than one can be supplied if comma separated.")
             .required(true)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     static final PropertyDescriptor TOPIC_TYPE = new PropertyDescriptor.Builder()
@@ -104,7 +105,7 @@ public class ConsumeKafkaRecord_0_10 extends AbstractProcessor {
         .displayName("Record Reader")
         .description("The Record Reader to use for incoming FlowFiles")
         .identifiesControllerService(RecordReaderFactory.class)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(true)
         .build();
 
@@ -113,7 +114,7 @@ public class ConsumeKafkaRecord_0_10 extends AbstractProcessor {
         .displayName("Record Writer")
         .description("The Record Writer to use in order to serialize the data before sending to Kafka")
         .identifiesControllerService(RecordSetWriterFactory.class)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(true)
         .build();
 
@@ -123,7 +124,7 @@ public class ConsumeKafkaRecord_0_10 extends AbstractProcessor {
             .description("A Group ID is used to identify consumers that are within the same consumer group. Corresponds to Kafka's 'group.id' property.")
             .required(true)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
 
     static final PropertyDescriptor AUTO_OFFSET_RESET = new PropertyDescriptor.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_10.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_10.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_10.java
index f6425de..cab15de 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_10.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_10.java
@@ -43,6 +43,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -87,7 +88,7 @@ public class ConsumeKafka_0_10 extends AbstractProcessor {
             .description("The name of the Kafka Topic(s) to pull from. More than one can be supplied if comma separated.")
             .required(true)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     static final PropertyDescriptor TOPIC_TYPE = new PropertyDescriptor.Builder()
@@ -105,7 +106,7 @@ public class ConsumeKafka_0_10 extends AbstractProcessor {
             .description("A Group ID is used to identify consumers that are within the same consumer group. Corresponds to Kafka's 'group.id' property.")
             .required(true)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
 
     static final PropertyDescriptor AUTO_OFFSET_RESET = new PropertyDescriptor.Builder()
@@ -132,7 +133,7 @@ public class ConsumeKafka_0_10 extends AbstractProcessor {
             .displayName("Message Demarcator")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .description("Since KafkaConsumer receives messages in batches, you have an option to output FlowFiles which contains "
                     + "all Kafka messages in a single batch for a given topic and partition and this property allows you to provide a string (interpreted as UTF-8) to use "
                     + "for demarcating apart multiple Kafka messages. This is an optional property and if not provided each Kafka message received "

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
index d835607..5b65b0d 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
@@ -41,6 +41,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.kerberos.KerberosCredentialsService;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.util.StandardValidators;
@@ -76,7 +77,7 @@ final class KafkaProcessorUtils {
             .description("A comma-separated list of known Kafka Brokers in the format <host>:<port>")
             .required(true)
             .addValidator(StandardValidators.HOSTNAME_PORT_LIST_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue("localhost:9092")
             .build();
     static final PropertyDescriptor SECURITY_PROTOCOL = new PropertyDescriptor.Builder()
@@ -84,7 +85,7 @@ final class KafkaProcessorUtils {
             .displayName("Security Protocol")
             .description("Protocol used to communicate with brokers. Corresponds to Kafka's 'security.protocol' property.")
             .required(true)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .allowableValues(SEC_PLAINTEXT, SEC_SSL, SEC_SASL_PLAINTEXT, SEC_SASL_SSL)
             .defaultValue(SEC_PLAINTEXT.getValue())
             .build();
@@ -96,7 +97,7 @@ final class KafkaProcessorUtils {
                     + "It is ignored unless one of the SASL options of the <Security Protocol> are selected.")
             .required(false)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
     static final PropertyDescriptor USER_PRINCIPAL = new PropertyDescriptor.Builder()
             .name("sasl.kerberos.principal")
@@ -105,7 +106,7 @@ final class KafkaProcessorUtils {
                     + "in the JVM properties defined in the bootstrap.conf file. This principal will be set into 'sasl.jaas.config' Kafka's property.")
             .required(false)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
     static final PropertyDescriptor USER_KEYTAB = new PropertyDescriptor.Builder()
             .name("sasl.kerberos.keytab")
@@ -114,7 +115,7 @@ final class KafkaProcessorUtils {
                     + "in the JVM properties defined in the bootstrap.conf file. This principal will be set into 'sasl.jaas.config' Kafka's property.")
             .required(false)
             .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
     static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
             .name("ssl.context.service")

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_10.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_10.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_10.java
index b86f60b..e670c7d 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_10.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_10.java
@@ -43,6 +43,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.DataUnit;
@@ -104,7 +105,7 @@ public class PublishKafkaRecord_0_10 extends AbstractProcessor {
         .description("The name of the Kafka Topic to publish to.")
         .required(true)
         .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .build();
 
     static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
@@ -112,7 +113,7 @@ public class PublishKafkaRecord_0_10 extends AbstractProcessor {
         .displayName("Record Reader")
         .description("The Record Reader to use for incoming FlowFiles")
         .identifiesControllerService(RecordReaderFactory.class)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(true)
         .build();
 
@@ -121,7 +122,7 @@ public class PublishKafkaRecord_0_10 extends AbstractProcessor {
         .displayName("Record Writer")
         .description("The Record Writer to use in order to serialize the data before sending to Kafka")
         .identifiesControllerService(RecordSetWriterFactory.class)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(true)
         .build();
 
@@ -130,7 +131,7 @@ public class PublishKafkaRecord_0_10 extends AbstractProcessor {
         .displayName("Message Key Field")
         .description("The name of a field in the Input Records that should be used as the Key for the Kafka message.")
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .required(false)
         .build();
 
@@ -139,7 +140,7 @@ public class PublishKafkaRecord_0_10 extends AbstractProcessor {
         .displayName("Delivery Guarantee")
         .description("Specifies the requirement for guaranteeing that a message is sent to Kafka. Corresponds to Kafka's 'acks' property.")
         .required(true)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues(DELIVERY_BEST_EFFORT, DELIVERY_ONE_NODE, DELIVERY_REPLICATED)
         .defaultValue(DELIVERY_BEST_EFFORT.getValue())
         .build();
@@ -151,7 +152,7 @@ public class PublishKafkaRecord_0_10 extends AbstractProcessor {
             + "entire 'send' call. Corresponds to Kafka's 'max.block.ms' property")
         .required(true)
         .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .defaultValue("5 sec")
         .build();
 
@@ -161,7 +162,7 @@ public class PublishKafkaRecord_0_10 extends AbstractProcessor {
         .description("After sending a message to Kafka, this indicates the amount of time that we are willing to wait for a response from Kafka. "
             + "If Kafka does not acknowledge the message within this time period, the FlowFile will be routed to 'failure'.")
         .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(true)
         .defaultValue("5 secs")
         .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_0_10.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_0_10.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_0_10.java
index c9d0f37..5a2716a 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_0_10.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_0_10.java
@@ -45,6 +45,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.DataUnit;
@@ -100,7 +101,7 @@ public class PublishKafka_0_10 extends AbstractProcessor {
         .description("The name of the Kafka Topic to publish to.")
         .required(true)
         .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .build();
 
     static final PropertyDescriptor DELIVERY_GUARANTEE = new PropertyDescriptor.Builder()
@@ -108,7 +109,7 @@ public class PublishKafka_0_10 extends AbstractProcessor {
         .displayName("Delivery Guarantee")
         .description("Specifies the requirement for guaranteeing that a message is sent to Kafka. Corresponds to Kafka's 'acks' property.")
         .required(true)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues(DELIVERY_BEST_EFFORT, DELIVERY_ONE_NODE, DELIVERY_REPLICATED)
         .defaultValue(DELIVERY_BEST_EFFORT.getValue())
         .build();
@@ -120,7 +121,7 @@ public class PublishKafka_0_10 extends AbstractProcessor {
             + "entire 'send' call. Corresponds to Kafka's 'max.block.ms' property")
         .required(true)
         .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .defaultValue("5 sec")
         .build();
 
@@ -130,7 +131,7 @@ public class PublishKafka_0_10 extends AbstractProcessor {
         .description("After sending a message to Kafka, this indicates the amount of time that we are willing to wait for a response from Kafka. "
             + "If Kafka does not acknowledge the message within this time period, the FlowFile will be routed to 'failure'.")
         .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(true)
         .defaultValue("5 secs")
         .build();
@@ -154,7 +155,7 @@ public class PublishKafka_0_10 extends AbstractProcessor {
             + "data loss on Kafka. During a topic compaction on Kafka, messages will be deduplicated based on this key.")
         .required(false)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .build();
 
     static final PropertyDescriptor KEY_ATTRIBUTE_ENCODING = new PropertyDescriptor.Builder()
@@ -171,7 +172,7 @@ public class PublishKafka_0_10 extends AbstractProcessor {
         .displayName("Message Demarcator")
         .required(false)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .description("Specifies the string (interpreted as UTF-8) to use for demarcating multiple messages within "
             + "a single FlowFile. If not specified, the entire content of the FlowFile will be used as a single message. If specified, the "
             + "contents of the FlowFile will be split on this delimiter and each section sent as a separate Kafka message. "

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_0_11.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_0_11.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_0_11.java
index c4b0920..2b202df 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_0_11.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_0_11.java
@@ -45,6 +45,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -89,7 +90,7 @@ public class ConsumeKafkaRecord_0_11 extends AbstractProcessor {
             .description("The name of the Kafka Topic(s) to pull from. More than one can be supplied if comma separated.")
             .required(true)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     static final PropertyDescriptor TOPIC_TYPE = new PropertyDescriptor.Builder()
@@ -106,7 +107,7 @@ public class ConsumeKafkaRecord_0_11 extends AbstractProcessor {
         .displayName("Record Reader")
         .description("The Record Reader to use for incoming FlowFiles")
         .identifiesControllerService(RecordReaderFactory.class)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(true)
         .build();
 
@@ -115,7 +116,7 @@ public class ConsumeKafkaRecord_0_11 extends AbstractProcessor {
         .displayName("Record Writer")
         .description("The Record Writer to use in order to serialize the data before sending to Kafka")
         .identifiesControllerService(RecordSetWriterFactory.class)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(true)
         .build();
 
@@ -125,7 +126,7 @@ public class ConsumeKafkaRecord_0_11 extends AbstractProcessor {
             .description("A Group ID is used to identify consumers that are within the same consumer group. Corresponds to Kafka's 'group.id' property.")
             .required(true)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
 
     static final PropertyDescriptor AUTO_OFFSET_RESET = new PropertyDescriptor.Builder()
@@ -167,7 +168,7 @@ public class ConsumeKafkaRecord_0_11 extends AbstractProcessor {
             + "read_uncomitted. This means that messages will be received as soon as they are written to Kafka but will be pulled, even if the producer cancels the transactions. If "
             + "this value is true, NiFi will not receive any messages for which the producer's transaction was canceled, but this can result in some latency since the consumer must wait "
             + "for the producer to finish its entire transaction instead of pulling as the messages become available.")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues("true", "false")
         .defaultValue("true")
         .required(true)
@@ -191,7 +192,7 @@ public class ConsumeKafkaRecord_0_11 extends AbstractProcessor {
             + "\".*\" if messages are expected to have header values that are unique per message, such as an identifier or timestamp, because it will prevent NiFi from bundling "
             + "the messages together efficiently.")
         .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(false)
         .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_11.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_11.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_11.java
index a8d0457..29c9ee3 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_11.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_11.java
@@ -44,6 +44,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -88,7 +89,7 @@ public class ConsumeKafka_0_11 extends AbstractProcessor {
             .description("The name of the Kafka Topic(s) to pull from. More than one can be supplied if comma separated.")
             .required(true)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     static final PropertyDescriptor TOPIC_TYPE = new PropertyDescriptor.Builder()
@@ -106,7 +107,7 @@ public class ConsumeKafka_0_11 extends AbstractProcessor {
             .description("A Group ID is used to identify consumers that are within the same consumer group. Corresponds to Kafka's 'group.id' property.")
             .required(true)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
 
     static final PropertyDescriptor AUTO_OFFSET_RESET = new PropertyDescriptor.Builder()
@@ -133,7 +134,7 @@ public class ConsumeKafka_0_11 extends AbstractProcessor {
             .displayName("Message Demarcator")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .description("Since KafkaConsumer receives messages in batches, you have an option to output FlowFiles which contains "
                     + "all Kafka messages in a single batch for a given topic and partition and this property allows you to provide a string (interpreted as UTF-8) to use "
                     + "for demarcating apart multiple Kafka messages. This is an optional property and if not provided each Kafka message received "
@@ -150,7 +151,7 @@ public class ConsumeKafka_0_11 extends AbstractProcessor {
             + "\".*\" if messages are expected to have header values that are unique per message, such as an identifier or timestamp, because it will prevent NiFi from bundling "
             + "the messages together efficiently.")
         .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(false)
         .build();
 
@@ -184,7 +185,7 @@ public class ConsumeKafka_0_11 extends AbstractProcessor {
             + "read_uncomitted. This means that messages will be received as soon as they are written to Kafka but will be pulled, even if the producer cancels the transactions. If "
             + "this value is true, NiFi will not receive any messages for which the producer's transaction was canceled, but this can result in some latency since the consumer must wait "
             + "for the producer to finish its entire transaction instead of pulling as the messages become available.")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues("true", "false")
         .defaultValue("true")
         .required(true)

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
index e88f3da..fa7ebcb 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
@@ -41,6 +41,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.kerberos.KerberosCredentialsService;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.util.StandardValidators;
@@ -76,7 +77,7 @@ final class KafkaProcessorUtils {
             .description("A comma-separated list of known Kafka Brokers in the format <host>:<port>")
             .required(true)
             .addValidator(StandardValidators.HOSTNAME_PORT_LIST_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue("localhost:9092")
             .build();
     static final PropertyDescriptor SECURITY_PROTOCOL = new PropertyDescriptor.Builder()
@@ -84,7 +85,7 @@ final class KafkaProcessorUtils {
             .displayName("Security Protocol")
             .description("Protocol used to communicate with brokers. Corresponds to Kafka's 'security.protocol' property.")
             .required(true)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .allowableValues(SEC_PLAINTEXT, SEC_SSL, SEC_SASL_PLAINTEXT, SEC_SASL_SSL)
             .defaultValue(SEC_PLAINTEXT.getValue())
             .build();
@@ -96,7 +97,7 @@ final class KafkaProcessorUtils {
                     + "It is ignored unless one of the SASL options of the <Security Protocol> are selected.")
             .required(false)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
     static final PropertyDescriptor USER_PRINCIPAL = new PropertyDescriptor.Builder()
             .name("sasl.kerberos.principal")
@@ -105,7 +106,7 @@ final class KafkaProcessorUtils {
                     + "in the JVM properties defined in the bootstrap.conf file. This principal will be set into 'sasl.jaas.config' Kafka's property.")
             .required(false)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
     static final PropertyDescriptor USER_KEYTAB = new PropertyDescriptor.Builder()
             .name("sasl.kerberos.keytab")
@@ -114,7 +115,7 @@ final class KafkaProcessorUtils {
                     + "in the JVM properties defined in the bootstrap.conf file. This principal will be set into 'sasl.jaas.config' Kafka's property.")
             .required(false)
             .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
     static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
             .name("ssl.context.service")

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_11.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_11.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_11.java
index 1521bfa..8bbec17 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_11.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_11.java
@@ -46,6 +46,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.DataUnit;
@@ -107,7 +108,7 @@ public class PublishKafkaRecord_0_11 extends AbstractProcessor {
         .description("The name of the Kafka Topic to publish to.")
         .required(true)
         .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .build();
 
     static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
@@ -115,7 +116,7 @@ public class PublishKafkaRecord_0_11 extends AbstractProcessor {
         .displayName("Record Reader")
         .description("The Record Reader to use for incoming FlowFiles")
         .identifiesControllerService(RecordReaderFactory.class)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(true)
         .build();
 
@@ -124,7 +125,7 @@ public class PublishKafkaRecord_0_11 extends AbstractProcessor {
         .displayName("Record Writer")
         .description("The Record Writer to use in order to serialize the data before sending to Kafka")
         .identifiesControllerService(RecordSetWriterFactory.class)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(true)
         .build();
 
@@ -133,7 +134,7 @@ public class PublishKafkaRecord_0_11 extends AbstractProcessor {
         .displayName("Message Key Field")
         .description("The name of a field in the Input Records that should be used as the Key for the Kafka message.")
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .required(false)
         .build();
 
@@ -142,7 +143,7 @@ public class PublishKafkaRecord_0_11 extends AbstractProcessor {
         .displayName("Delivery Guarantee")
         .description("Specifies the requirement for guaranteeing that a message is sent to Kafka. Corresponds to Kafka's 'acks' property.")
         .required(true)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues(DELIVERY_BEST_EFFORT, DELIVERY_ONE_NODE, DELIVERY_REPLICATED)
         .defaultValue(DELIVERY_BEST_EFFORT.getValue())
         .build();
@@ -154,7 +155,7 @@ public class PublishKafkaRecord_0_11 extends AbstractProcessor {
             + "entire 'send' call. Corresponds to Kafka's 'max.block.ms' property")
         .required(true)
         .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .defaultValue("5 sec")
         .build();
 
@@ -164,7 +165,7 @@ public class PublishKafkaRecord_0_11 extends AbstractProcessor {
         .description("After sending a message to Kafka, this indicates the amount of time that we are willing to wait for a response from Kafka. "
             + "If Kafka does not acknowledge the message within this time period, the FlowFile will be routed to 'failure'.")
         .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(true)
         .defaultValue("5 secs")
         .build();
@@ -204,7 +205,7 @@ public class PublishKafkaRecord_0_11 extends AbstractProcessor {
             + "Any attribute whose name matches the regex will be added to the Kafka messages as a Header. "
             + "If not specified, no FlowFile attributes will be added as headers.")
         .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(false)
         .build();
     static final PropertyDescriptor USE_TRANSACTIONS = new PropertyDescriptor.Builder()
@@ -214,7 +215,7 @@ public class PublishKafkaRecord_0_11 extends AbstractProcessor {
             + "and this property is set to false, then the messages that have already been sent to Kafka will continue on and be delivered to consumers. "
             + "If this is set to true, then the Kafka transaction will be rolled back so that those messages are not available to consumers. Setting this to true "
             + "requires that the <Delivery Guarantee> property be set to \"Guarantee Replicated Delivery.\"")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues("true", "false")
         .defaultValue("true")
         .required(true)

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_0_11.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_0_11.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_0_11.java
index 130954a..5efe913 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_0_11.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_0_11.java
@@ -47,6 +47,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.DataUnit;
@@ -102,7 +103,7 @@ public class PublishKafka_0_11 extends AbstractProcessor {
         .description("The name of the Kafka Topic to publish to.")
         .required(true)
         .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .build();
 
     static final PropertyDescriptor DELIVERY_GUARANTEE = new PropertyDescriptor.Builder()
@@ -110,7 +111,7 @@ public class PublishKafka_0_11 extends AbstractProcessor {
         .displayName("Delivery Guarantee")
         .description("Specifies the requirement for guaranteeing that a message is sent to Kafka. Corresponds to Kafka's 'acks' property.")
         .required(true)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues(DELIVERY_BEST_EFFORT, DELIVERY_ONE_NODE, DELIVERY_REPLICATED)
         .defaultValue(DELIVERY_BEST_EFFORT.getValue())
         .build();
@@ -122,7 +123,7 @@ public class PublishKafka_0_11 extends AbstractProcessor {
             + "entire 'send' call. Corresponds to Kafka's 'max.block.ms' property")
         .required(true)
         .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .defaultValue("5 sec")
         .build();
 
@@ -132,7 +133,7 @@ public class PublishKafka_0_11 extends AbstractProcessor {
         .description("After sending a message to Kafka, this indicates the amount of time that we are willing to wait for a response from Kafka. "
             + "If Kafka does not acknowledge the message within this time period, the FlowFile will be routed to 'failure'.")
         .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(true)
         .defaultValue("5 secs")
         .build();
@@ -156,7 +157,7 @@ public class PublishKafka_0_11 extends AbstractProcessor {
             + "data loss on Kafka. During a topic compaction on Kafka, messages will be deduplicated based on this key.")
         .required(false)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .build();
 
     static final PropertyDescriptor KEY_ATTRIBUTE_ENCODING = new PropertyDescriptor.Builder()
@@ -173,7 +174,7 @@ public class PublishKafka_0_11 extends AbstractProcessor {
         .displayName("Message Demarcator")
         .required(false)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .description("Specifies the string (interpreted as UTF-8) to use for demarcating multiple messages within "
             + "a single FlowFile. If not specified, the entire content of the FlowFile will be used as a single message. If specified, the "
             + "contents of the FlowFile will be split on this delimiter and each section sent as a separate Kafka message. "
@@ -206,7 +207,7 @@ public class PublishKafka_0_11 extends AbstractProcessor {
             + "Any attribute whose name matches the regex will be added to the Kafka messages as a Header. "
             + "If not specified, no FlowFile attributes will be added as headers.")
         .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(false)
         .build();
     static final PropertyDescriptor USE_TRANSACTIONS = new PropertyDescriptor.Builder()
@@ -216,7 +217,7 @@ public class PublishKafka_0_11 extends AbstractProcessor {
             + "and this property is set to false, then the messages that have already been sent to Kafka will continue on and be delivered to consumers. "
             + "If this is set to true, then the Kafka transaction will be rolled back so that those messages are not available to consumers. Setting this to true "
             + "requires that the <Delivery Guarantee> property be set to \"Guarantee Replicated Delivery.\"")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues("true", "false")
         .defaultValue("true")
         .required(true)

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java
index b1eadcf..e01afdd 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java
@@ -50,6 +50,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -93,14 +94,14 @@ public class GetKafka extends AbstractProcessor {
                     + " combinations. For example, host1:2181,host2:2181,host3:2188")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
     public static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder()
             .name("Topic Name")
             .description("The Kafka Topic to pull messages from")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
     public static final PropertyDescriptor ZOOKEEPER_COMMIT_DELAY = new PropertyDescriptor.Builder()
             .name("Zookeeper Commit Frequency")
@@ -108,7 +109,7 @@ public class GetKafka extends AbstractProcessor {
                     + " result in better overall performance but can result in more data duplication if a NiFi node is lost")
             .required(true)
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .defaultValue("60 secs")
             .build();
     public static final PropertyDescriptor ZOOKEEPER_TIMEOUT = new PropertyDescriptor.Builder()
@@ -116,7 +117,7 @@ public class GetKafka extends AbstractProcessor {
             .description("The amount of time to wait for a response from ZooKeeper before determining that there is a communications error")
             .required(true)
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .defaultValue("30 secs")
             .build();
     public static final PropertyDescriptor KAFKA_TIMEOUT = new PropertyDescriptor.Builder()
@@ -124,7 +125,7 @@ public class GetKafka extends AbstractProcessor {
             .description("The amount of time to wait for a response from Kafka before determining that there is a communications error")
             .required(true)
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .defaultValue("30 secs")
             .build();
     public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
@@ -134,7 +135,7 @@ public class GetKafka extends AbstractProcessor {
                     + "If the messages from Kafka should not be concatenated together, leave this value at 1.")
             .required(true)
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .defaultValue("1")
             .build();
     public static final PropertyDescriptor MESSAGE_DEMARCATOR = new PropertyDescriptor.Builder()
@@ -144,7 +145,7 @@ public class GetKafka extends AbstractProcessor {
                     + "this value will be placed in between them.")
             .required(true)
             .addValidator(Validator.VALID) // accept anything as a demarcator, including empty string
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .defaultValue("\\n")
             .build();
 
@@ -153,14 +154,14 @@ public class GetKafka extends AbstractProcessor {
             .description("Client Name to use when communicating with Kafka")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
     public static final PropertyDescriptor GROUP_ID = new PropertyDescriptor.Builder()
             .name("Group ID")
             .description("A Group ID is used to identify consumers that are within the same consumer group")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor AUTO_OFFSET_RESET = new PropertyDescriptor.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
index ab24e57..be93736 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
@@ -38,6 +38,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.ProcessContext;
@@ -112,14 +113,14 @@ public class PutKafka extends AbstractKafkaProcessor<KafkaPublisher> {
             .description("A comma-separated list of known Kafka Brokers in the format <host>:<port>")
             .required(true)
             .addValidator(StandardValidators.HOSTNAME_PORT_LIST_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
     public static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder()
             .name("Topic Name")
             .description("The Kafka Topic of interest")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     /**
      * @deprecated Kafka 0.8.x producer doesn't use 'partitioner.class' property.
@@ -137,7 +138,7 @@ public class PutKafka extends AbstractKafkaProcessor<KafkaPublisher> {
                             + "in the same FlowFile will be sent to the same partition. If a partition is specified but is not valid, "
                             + "then the FlowFile will be routed to failure relationship.")
             .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(false)
             .build();
     public static final PropertyDescriptor KEY = new PropertyDescriptor.Builder()
@@ -145,12 +146,12 @@ public class PutKafka extends AbstractKafkaProcessor<KafkaPublisher> {
             .description("The Key to use for the Message")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     public static final PropertyDescriptor DELIVERY_GUARANTEE = new PropertyDescriptor.Builder()
             .name("Delivery Guarantee")
             .description("Specifies the requirement for guaranteeing that a message is sent to Kafka").required(true)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .allowableValues(DELIVERY_BEST_EFFORT, DELIVERY_ONE_NODE, DELIVERY_REPLICATED)
             .defaultValue(DELIVERY_BEST_EFFORT.getValue())
             .build();
@@ -165,14 +166,14 @@ public class PutKafka extends AbstractKafkaProcessor<KafkaPublisher> {
                             + "sent to the 'failure' relationship.")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     public static final PropertyDescriptor MAX_BUFFER_SIZE = new PropertyDescriptor.Builder()
             .name("Max Buffer Size")
             .description("The maximum amount of data to buffer in memory before sending to Kafka")
             .required(true)
             .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .defaultValue("5 MB")
             .build();
     static final PropertyDescriptor MAX_RECORD_SIZE = new PropertyDescriptor.Builder()
@@ -187,14 +188,14 @@ public class PutKafka extends AbstractKafkaProcessor<KafkaPublisher> {
             .description("The amount of time to wait for a response from Kafka before determining that there is a communications error")
             .required(true)
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .defaultValue("30 secs").build();
     public static final PropertyDescriptor CLIENT_NAME = new PropertyDescriptor.Builder()
             .name("Client Name")
             .description("Client Name to use when communicating with Kafka")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
     public static final PropertyDescriptor BATCH_NUM_MESSAGES = new PropertyDescriptor.Builder()
             .name("Async Batch Size")


[07/15] nifi git commit: NIFI-4149 - Indicate if EL is evaluated against FFs or not - take into account input requirement for documentation rendering - Renamed variable registry scope and added comments - Doc + change in mock framework to check scope + u

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java
index d02d011..905b7fc 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java
@@ -46,6 +46,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -88,7 +89,7 @@ public class ConsumeKafka extends AbstractProcessor {
             .description("The name of the Kafka Topic(s) to pull from. More than one can be supplied if comma separated.")
             .required(true)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     static final PropertyDescriptor GROUP_ID = new PropertyDescriptor.Builder()
@@ -97,7 +98,7 @@ public class ConsumeKafka extends AbstractProcessor {
             .description("A Group ID is used to identify consumers that are within the same consumer group. Corresponds to Kafka's 'group.id' property.")
             .required(true)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
 
     static final PropertyDescriptor AUTO_OFFSET_RESET = new PropertyDescriptor.Builder()
@@ -124,7 +125,7 @@ public class ConsumeKafka extends AbstractProcessor {
             .displayName("Message Demarcator")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .description("Since KafkaConsumer receives messages in batches, you have an option to output FlowFiles which contains "
                     + "all Kafka messages in a single batch for a given topic and partition and this property allows you to provide a string (interpreted as UTF-8) to use "
                     + "for demarcating apart multiple Kafka messages. This is an optional property and if not provided each Kafka message received "

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
index cbe2e24..77a7f21 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
@@ -41,6 +41,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.ssl.SSLContextService;
@@ -74,7 +75,7 @@ final class KafkaProcessorUtils {
             .description("A comma-separated list of known Kafka Brokers in the format <host>:<port>")
             .required(true)
             .addValidator(StandardValidators.HOSTNAME_PORT_LIST_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue("localhost:9092")
             .build();
     static final PropertyDescriptor SECURITY_PROTOCOL = new PropertyDescriptor.Builder()
@@ -82,7 +83,7 @@ final class KafkaProcessorUtils {
             .displayName("Security Protocol")
             .description("Protocol used to communicate with brokers. Corresponds to Kafka's 'security.protocol' property.")
             .required(true)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .allowableValues(SEC_PLAINTEXT, SEC_SSL, SEC_SASL_PLAINTEXT, SEC_SASL_SSL)
             .defaultValue(SEC_PLAINTEXT.getValue())
             .build();
@@ -94,7 +95,7 @@ final class KafkaProcessorUtils {
                     + "It is ignored unless one of the SASL options of the <Security Protocol> are selected.")
             .required(false)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
     static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
             .name("ssl.context.service")

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka.java
index 32d1ea7..0430f0c 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka.java
@@ -45,6 +45,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.DataUnit;
@@ -102,7 +103,7 @@ public class PublishKafka extends AbstractProcessor {
         .description("The name of the Kafka Topic to publish to.")
         .required(true)
         .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .build();
 
     static final PropertyDescriptor DELIVERY_GUARANTEE = new PropertyDescriptor.Builder()
@@ -110,7 +111,7 @@ public class PublishKafka extends AbstractProcessor {
         .displayName("Delivery Guarantee")
         .description("Specifies the requirement for guaranteeing that a message is sent to Kafka. Corresponds to Kafka's 'acks' property.")
         .required(true)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues(DELIVERY_BEST_EFFORT, DELIVERY_ONE_NODE, DELIVERY_REPLICATED)
         .defaultValue(DELIVERY_BEST_EFFORT.getValue())
         .build();
@@ -122,7 +123,7 @@ public class PublishKafka extends AbstractProcessor {
             + "entire 'send' call. Corresponds to Kafka's 'max.block.ms' property")
         .required(true)
         .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .defaultValue("5 sec")
         .build();
 
@@ -132,7 +133,7 @@ public class PublishKafka extends AbstractProcessor {
         .description("After sending a message to Kafka, this indicates the amount of time that we are willing to wait for a response from Kafka. "
             + "If Kafka does not acknowledge the message within this time period, the FlowFile will be routed to 'failure'.")
         .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(true)
         .defaultValue("5 secs")
         .build();
@@ -156,7 +157,7 @@ public class PublishKafka extends AbstractProcessor {
             + "data loss on Kafka. During a topic compaction on Kafka, messages will be deduplicated based on this key.")
         .required(false)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .build();
 
     static final PropertyDescriptor KEY_ATTRIBUTE_ENCODING = new PropertyDescriptor.Builder()
@@ -173,7 +174,7 @@ public class PublishKafka extends AbstractProcessor {
         .displayName("Message Demarcator")
         .required(false)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .description("Specifies the string (interpreted as UTF-8) to use for demarcating multiple messages within "
             + "a single FlowFile. If not specified, the entire content of the FlowFile will be used as a single message. If specified, the "
             + "contents of the FlowFile will be split on this delimiter and each section sent as a separate Kafka message. "

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_1_0.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_1_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_1_0.java
index a64cb8e..efcc242 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_1_0.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_1_0.java
@@ -45,6 +45,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -89,7 +90,7 @@ public class ConsumeKafkaRecord_1_0 extends AbstractProcessor {
             .description("The name of the Kafka Topic(s) to pull from. More than one can be supplied if comma separated.")
             .required(true)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     static final PropertyDescriptor TOPIC_TYPE = new PropertyDescriptor.Builder()
@@ -106,7 +107,7 @@ public class ConsumeKafkaRecord_1_0 extends AbstractProcessor {
         .displayName("Record Reader")
         .description("The Record Reader to use for incoming FlowFiles")
         .identifiesControllerService(RecordReaderFactory.class)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(true)
         .build();
 
@@ -115,7 +116,7 @@ public class ConsumeKafkaRecord_1_0 extends AbstractProcessor {
         .displayName("Record Writer")
         .description("The Record Writer to use in order to serialize the data before sending to Kafka")
         .identifiesControllerService(RecordSetWriterFactory.class)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(true)
         .build();
 
@@ -125,7 +126,7 @@ public class ConsumeKafkaRecord_1_0 extends AbstractProcessor {
             .description("A Group ID is used to identify consumers that are within the same consumer group. Corresponds to Kafka's 'group.id' property.")
             .required(true)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
 
     static final PropertyDescriptor AUTO_OFFSET_RESET = new PropertyDescriptor.Builder()
@@ -167,7 +168,7 @@ public class ConsumeKafkaRecord_1_0 extends AbstractProcessor {
             + "read_uncomitted. This means that messages will be received as soon as they are written to Kafka but will be pulled, even if the producer cancels the transactions. If "
             + "this value is true, NiFi will not receive any messages for which the producer's transaction was canceled, but this can result in some latency since the consumer must wait "
             + "for the producer to finish its entire transaction instead of pulling as the messages become available.")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues("true", "false")
         .defaultValue("true")
         .required(true)
@@ -191,7 +192,7 @@ public class ConsumeKafkaRecord_1_0 extends AbstractProcessor {
             + "\".*\" if messages are expected to have header values that are unique per message, such as an identifier or timestamp, because it will prevent NiFi from bundling "
             + "the messages together efficiently.")
         .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(false)
         .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_1_0.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_1_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_1_0.java
index fdc2cb3..ff6a250 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_1_0.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_1_0.java
@@ -44,6 +44,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -88,7 +89,7 @@ public class ConsumeKafka_1_0 extends AbstractProcessor {
             .description("The name of the Kafka Topic(s) to pull from. More than one can be supplied if comma separated.")
             .required(true)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     static final PropertyDescriptor TOPIC_TYPE = new PropertyDescriptor.Builder()
@@ -106,7 +107,7 @@ public class ConsumeKafka_1_0 extends AbstractProcessor {
             .description("A Group ID is used to identify consumers that are within the same consumer group. Corresponds to Kafka's 'group.id' property.")
             .required(true)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
 
     static final PropertyDescriptor AUTO_OFFSET_RESET = new PropertyDescriptor.Builder()
@@ -133,7 +134,7 @@ public class ConsumeKafka_1_0 extends AbstractProcessor {
             .displayName("Message Demarcator")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .description("Since KafkaConsumer receives messages in batches, you have an option to output FlowFiles which contains "
                     + "all Kafka messages in a single batch for a given topic and partition and this property allows you to provide a string (interpreted as UTF-8) to use "
                     + "for demarcating apart multiple Kafka messages. This is an optional property and if not provided each Kafka message received "
@@ -150,7 +151,7 @@ public class ConsumeKafka_1_0 extends AbstractProcessor {
             + "\".*\" if messages are expected to have header values that are unique per message, such as an identifier or timestamp, because it will prevent NiFi from bundling "
             + "the messages together efficiently.")
         .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(false)
         .build();
 
@@ -184,7 +185,7 @@ public class ConsumeKafka_1_0 extends AbstractProcessor {
             + "read_uncomitted. This means that messages will be received as soon as they are written to Kafka but will be pulled, even if the producer cancels the transactions. If "
             + "this value is true, NiFi will not receive any messages for which the producer's transaction was canceled, but this can result in some latency since the consumer must wait "
             + "for the producer to finish its entire transaction instead of pulling as the messages become available.")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues("true", "false")
         .defaultValue("true")
         .required(true)

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
index d835607..5b65b0d 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
@@ -41,6 +41,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.kerberos.KerberosCredentialsService;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.util.StandardValidators;
@@ -76,7 +77,7 @@ final class KafkaProcessorUtils {
             .description("A comma-separated list of known Kafka Brokers in the format <host>:<port>")
             .required(true)
             .addValidator(StandardValidators.HOSTNAME_PORT_LIST_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue("localhost:9092")
             .build();
     static final PropertyDescriptor SECURITY_PROTOCOL = new PropertyDescriptor.Builder()
@@ -84,7 +85,7 @@ final class KafkaProcessorUtils {
             .displayName("Security Protocol")
             .description("Protocol used to communicate with brokers. Corresponds to Kafka's 'security.protocol' property.")
             .required(true)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .allowableValues(SEC_PLAINTEXT, SEC_SSL, SEC_SASL_PLAINTEXT, SEC_SASL_SSL)
             .defaultValue(SEC_PLAINTEXT.getValue())
             .build();
@@ -96,7 +97,7 @@ final class KafkaProcessorUtils {
                     + "It is ignored unless one of the SASL options of the <Security Protocol> are selected.")
             .required(false)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
     static final PropertyDescriptor USER_PRINCIPAL = new PropertyDescriptor.Builder()
             .name("sasl.kerberos.principal")
@@ -105,7 +106,7 @@ final class KafkaProcessorUtils {
                     + "in the JVM properties defined in the bootstrap.conf file. This principal will be set into 'sasl.jaas.config' Kafka's property.")
             .required(false)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
     static final PropertyDescriptor USER_KEYTAB = new PropertyDescriptor.Builder()
             .name("sasl.kerberos.keytab")
@@ -114,7 +115,7 @@ final class KafkaProcessorUtils {
                     + "in the JVM properties defined in the bootstrap.conf file. This principal will be set into 'sasl.jaas.config' Kafka's property.")
             .required(false)
             .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
     static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
             .name("ssl.context.service")

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_1_0.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_1_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_1_0.java
index e26d665..d0f368d 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_1_0.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_1_0.java
@@ -46,6 +46,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.DataUnit;
@@ -107,7 +108,7 @@ public class PublishKafkaRecord_1_0 extends AbstractProcessor {
         .description("The name of the Kafka Topic to publish to.")
         .required(true)
         .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .build();
 
     static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
@@ -115,7 +116,7 @@ public class PublishKafkaRecord_1_0 extends AbstractProcessor {
         .displayName("Record Reader")
         .description("The Record Reader to use for incoming FlowFiles")
         .identifiesControllerService(RecordReaderFactory.class)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(true)
         .build();
 
@@ -124,7 +125,7 @@ public class PublishKafkaRecord_1_0 extends AbstractProcessor {
         .displayName("Record Writer")
         .description("The Record Writer to use in order to serialize the data before sending to Kafka")
         .identifiesControllerService(RecordSetWriterFactory.class)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(true)
         .build();
 
@@ -133,7 +134,7 @@ public class PublishKafkaRecord_1_0 extends AbstractProcessor {
         .displayName("Message Key Field")
         .description("The name of a field in the Input Records that should be used as the Key for the Kafka message.")
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .required(false)
         .build();
 
@@ -142,7 +143,7 @@ public class PublishKafkaRecord_1_0 extends AbstractProcessor {
         .displayName("Delivery Guarantee")
         .description("Specifies the requirement for guaranteeing that a message is sent to Kafka. Corresponds to Kafka's 'acks' property.")
         .required(true)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues(DELIVERY_BEST_EFFORT, DELIVERY_ONE_NODE, DELIVERY_REPLICATED)
         .defaultValue(DELIVERY_BEST_EFFORT.getValue())
         .build();
@@ -154,7 +155,7 @@ public class PublishKafkaRecord_1_0 extends AbstractProcessor {
             + "entire 'send' call. Corresponds to Kafka's 'max.block.ms' property")
         .required(true)
         .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .defaultValue("5 sec")
         .build();
 
@@ -164,7 +165,7 @@ public class PublishKafkaRecord_1_0 extends AbstractProcessor {
         .description("After sending a message to Kafka, this indicates the amount of time that we are willing to wait for a response from Kafka. "
             + "If Kafka does not acknowledge the message within this time period, the FlowFile will be routed to 'failure'.")
         .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(true)
         .defaultValue("5 secs")
         .build();
@@ -204,7 +205,7 @@ public class PublishKafkaRecord_1_0 extends AbstractProcessor {
             + "Any attribute whose name matches the regex will be added to the Kafka messages as a Header. "
             + "If not specified, no FlowFile attributes will be added as headers.")
         .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(false)
         .build();
     static final PropertyDescriptor USE_TRANSACTIONS = new PropertyDescriptor.Builder()
@@ -214,7 +215,7 @@ public class PublishKafkaRecord_1_0 extends AbstractProcessor {
             + "and this property is set to false, then the messages that have already been sent to Kafka will continue on and be delivered to consumers. "
             + "If this is set to true, then the Kafka transaction will be rolled back so that those messages are not available to consumers. Setting this to true "
             + "requires that the <Delivery Guarantee> property be set to \"Guarantee Replicated Delivery.\"")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues("true", "false")
         .defaultValue("true")
         .required(true)

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_1_0.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_1_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_1_0.java
index 48f7747..d7e910d 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_1_0.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_1_0.java
@@ -47,6 +47,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.DataUnit;
@@ -102,7 +103,7 @@ public class PublishKafka_1_0 extends AbstractProcessor {
         .description("The name of the Kafka Topic to publish to.")
         .required(true)
         .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .build();
 
     static final PropertyDescriptor DELIVERY_GUARANTEE = new PropertyDescriptor.Builder()
@@ -110,7 +111,7 @@ public class PublishKafka_1_0 extends AbstractProcessor {
         .displayName("Delivery Guarantee")
         .description("Specifies the requirement for guaranteeing that a message is sent to Kafka. Corresponds to Kafka's 'acks' property.")
         .required(true)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues(DELIVERY_BEST_EFFORT, DELIVERY_ONE_NODE, DELIVERY_REPLICATED)
         .defaultValue(DELIVERY_BEST_EFFORT.getValue())
         .build();
@@ -122,7 +123,7 @@ public class PublishKafka_1_0 extends AbstractProcessor {
             + "entire 'send' call. Corresponds to Kafka's 'max.block.ms' property")
         .required(true)
         .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .defaultValue("5 sec")
         .build();
 
@@ -132,7 +133,7 @@ public class PublishKafka_1_0 extends AbstractProcessor {
         .description("After sending a message to Kafka, this indicates the amount of time that we are willing to wait for a response from Kafka. "
             + "If Kafka does not acknowledge the message within this time period, the FlowFile will be routed to 'failure'.")
         .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(true)
         .defaultValue("5 secs")
         .build();
@@ -156,7 +157,7 @@ public class PublishKafka_1_0 extends AbstractProcessor {
             + "data loss on Kafka. During a topic compaction on Kafka, messages will be deduplicated based on this key.")
         .required(false)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .build();
 
     static final PropertyDescriptor KEY_ATTRIBUTE_ENCODING = new PropertyDescriptor.Builder()
@@ -173,7 +174,7 @@ public class PublishKafka_1_0 extends AbstractProcessor {
         .displayName("Message Demarcator")
         .required(false)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .description("Specifies the string (interpreted as UTF-8) to use for demarcating multiple messages within "
             + "a single FlowFile. If not specified, the entire content of the FlowFile will be used as a single message. If specified, the "
             + "contents of the FlowFile will be split on this delimiter and each section sent as a separate Kafka message. "
@@ -206,7 +207,7 @@ public class PublishKafka_1_0 extends AbstractProcessor {
             + "Any attribute whose name matches the regex will be added to the Kafka messages as a Header. "
             + "If not specified, no FlowFile attributes will be added as headers.")
         .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(false)
         .build();
     static final PropertyDescriptor USE_TRANSACTIONS = new PropertyDescriptor.Builder()
@@ -216,7 +217,7 @@ public class PublishKafka_1_0 extends AbstractProcessor {
             + "and this property is set to false, then the messages that have already been sent to Kafka will continue on and be delivered to consumers. "
             + "If this is set to true, then the Kafka transaction will be rolled back so that those messages are not available to consumers. Setting this to true "
             + "requires that the <Delivery Guarantee> property be set to \"Guarantee Replicated Delivery.\"")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues("true", "false")
         .defaultValue("true")
         .required(true)

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/AbstractKiteProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/AbstractKiteProcessor.java b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/AbstractKiteProcessor.java
index 32b83b8..65dcd5f 100644
--- a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/AbstractKiteProcessor.java
+++ b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/AbstractKiteProcessor.java
@@ -35,6 +35,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processors.hadoop.HadoopValidators;
@@ -56,7 +57,7 @@ abstract class AbstractKiteProcessor extends AbstractProcessor {
                     + "will search the classpath for a 'core-site.xml' and 'hdfs-site.xml' file or will revert to a default configuration.")
             .required(false)
             .addValidator(HadoopValidators.ONE_OR_MORE_FILE_EXISTS_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     protected static final Validator RECOGNIZED_URI = new Validator() {

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertAvroSchema.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertAvroSchema.java b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertAvroSchema.java
index a3fffc3..a3602e3 100644
--- a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertAvroSchema.java
+++ b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertAvroSchema.java
@@ -27,6 +27,7 @@ import java.util.List;
 import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.regex.Pattern;
 
 import org.apache.avro.Schema;
@@ -45,6 +46,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -61,7 +63,6 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
-import java.util.concurrent.atomic.AtomicLong;
 
 @Tags({ "avro", "convert", "kite" })
 @CapabilityDescription("Convert records from one Avro schema to another, including support for flattening and simple type conversions")
@@ -155,7 +156,7 @@ public class ConvertAvroSchema extends AbstractKiteConvertProcessor {
             .name("Input Schema")
             .description("Avro Schema of Input Flowfiles.  This can be a URI (dataset, view, or resource) or literal JSON schema.")
             .addValidator(SCHEMA_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(true)
             .build();
 
@@ -164,7 +165,7 @@ public class ConvertAvroSchema extends AbstractKiteConvertProcessor {
             .name("Output Schema")
             .description("Avro Schema of Output Flowfiles.  This can be a URI (dataset, view, or resource) or literal JSON schema.")
             .addValidator(MAPPED_SCHEMA_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(true).build();
 
     @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertCSVToAvro.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertCSVToAvro.java b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertCSVToAvro.java
index 9702916..68e2647 100644
--- a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertCSVToAvro.java
+++ b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertCSVToAvro.java
@@ -39,6 +39,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -102,7 +103,7 @@ public class ConvertCSVToAvro extends AbstractKiteConvertProcessor {
         .name("Record schema")
         .description("Outgoing Avro schema for each record created from a CSV row")
         .addValidator(SCHEMA_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .required(true)
         .build();
 
@@ -111,7 +112,7 @@ public class ConvertCSVToAvro extends AbstractKiteConvertProcessor {
         .name("CSV charset")
         .description("Character set for CSV files")
         .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .defaultValue(DEFAULTS.charset)
         .build();
 
@@ -120,7 +121,7 @@ public class ConvertCSVToAvro extends AbstractKiteConvertProcessor {
         .name("CSV delimiter")
         .description("Delimiter character for CSV records")
         .addValidator(CHAR_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .defaultValue(DEFAULTS.delimiter)
         .build();
 
@@ -129,7 +130,7 @@ public class ConvertCSVToAvro extends AbstractKiteConvertProcessor {
         .name("CSV quote character")
         .description("Quote character for CSV values")
         .addValidator(CHAR_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .defaultValue(DEFAULTS.quote)
         .build();
 
@@ -138,7 +139,7 @@ public class ConvertCSVToAvro extends AbstractKiteConvertProcessor {
         .name("CSV escape character")
         .description("Escape character for CSV values")
         .addValidator(CHAR_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .defaultValue(DEFAULTS.escape)
         .build();
 
@@ -147,7 +148,7 @@ public class ConvertCSVToAvro extends AbstractKiteConvertProcessor {
         .name("Use CSV header line")
         .description("Whether to use the first line as a header")
         .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .defaultValue(String.valueOf(DEFAULTS.useHeader))
         .build();
 
@@ -156,7 +157,7 @@ public class ConvertCSVToAvro extends AbstractKiteConvertProcessor {
         .name("Lines to skip")
         .description("Number of lines to skip before reading header or data")
         .addValidator(createLongValidator(0L, Integer.MAX_VALUE, true))
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .defaultValue(String.valueOf(DEFAULTS.linesToSkip))
         .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertJSONToAvro.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertJSONToAvro.java b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertJSONToAvro.java
index 1127a2d..f93e522 100644
--- a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertJSONToAvro.java
+++ b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertJSONToAvro.java
@@ -23,6 +23,7 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.List;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.avro.Schema;
 import org.apache.avro.file.DataFileWriter;
@@ -32,6 +33,7 @@ import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -48,7 +50,6 @@ import org.kitesdk.data.spi.filesystem.JSONFileReader;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
-import java.util.concurrent.atomic.AtomicLong;
 
 @Tags({"kite", "json", "avro"})
 @InputRequirement(Requirement.INPUT_REQUIRED)
@@ -76,7 +77,7 @@ public class ConvertJSONToAvro extends AbstractKiteConvertProcessor {
             .name("Record schema")
             .description("Outgoing Avro schema for each record created from a JSON object")
             .addValidator(SCHEMA_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(true)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/InferAvroSchema.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/InferAvroSchema.java b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/InferAvroSchema.java
index 0edbd2b..4344ce0 100644
--- a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/InferAvroSchema.java
+++ b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/InferAvroSchema.java
@@ -32,6 +32,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.ProcessContext;
@@ -154,7 +155,7 @@ public class InferAvroSchema
                     " as the underlying data. Setting this property will cause the value of" +
                     " \"" + GET_CSV_HEADER_DEFINITION_FROM_INPUT.getName() + "\" to be ignored instead using this value.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue(null)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
@@ -170,14 +171,14 @@ public class InferAvroSchema
                     " no data is skipped.")
             .required(true)
             .defaultValue("0")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
             .build();
 
     public static final PropertyDescriptor DELIMITER = new PropertyDescriptor.Builder()
             .name("CSV delimiter")
             .description("Delimiter character for CSV records")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(CHAR_VALIDATOR)
             .defaultValue(",")
             .build();
@@ -188,7 +189,7 @@ public class InferAvroSchema
                     " in the CSV FlowFile content data.")
             .required(true)
             .defaultValue("\\")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -198,7 +199,7 @@ public class InferAvroSchema
                     " character in the CSV FlowFile content data.")
             .required(true)
             .defaultValue("'")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -207,7 +208,7 @@ public class InferAvroSchema
             .description("Value to be placed in the Avro record schema \"name\" field. The value must adhere to the Avro naming "
                     + "rules for fullname. If Expression Language is present then the evaluated value must adhere to the Avro naming rules.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.createRegexMatchingValidator(AVRO_RECORD_NAME_PATTERN))
             .build();
 
@@ -216,7 +217,7 @@ public class InferAvroSchema
             .description("Character encoding of CSV data.")
             .required(true)
             .defaultValue("UTF-8")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
             .build();
 
@@ -236,7 +237,7 @@ public class InferAvroSchema
                     " the appropriate type. However the default value of 10 is almost always enough.")
             .required(true)
             .defaultValue("10")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/StoreInKiteDataset.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/StoreInKiteDataset.java b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/StoreInKiteDataset.java
index 1a39664..7730554 100644
--- a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/StoreInKiteDataset.java
+++ b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/StoreInKiteDataset.java
@@ -32,6 +32,7 @@ import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -76,7 +77,7 @@ public class StoreInKiteDataset extends AbstractKiteProcessor {
             .name("Target dataset URI")
             .description("URI that identifies a Kite dataset where data will be stored")
             .addValidator(RECOGNIZED_URI)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(true)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-kudu-bundle/nifi-kudu-processors/src/main/java/org/apache/nifi/processors/kudu/AbstractKudu.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kudu-bundle/nifi-kudu-processors/src/main/java/org/apache/nifi/processors/kudu/AbstractKudu.java b/nifi-nar-bundles/nifi-kudu-bundle/nifi-kudu-processors/src/main/java/org/apache/nifi/processors/kudu/AbstractKudu.java
index 359e817..68e7004 100644
--- a/nifi-nar-bundles/nifi-kudu-bundle/nifi-kudu-processors/src/main/java/org/apache/nifi/processors/kudu/AbstractKudu.java
+++ b/nifi-nar-bundles/nifi-kudu-bundle/nifi-kudu-processors/src/main/java/org/apache/nifi/processors/kudu/AbstractKudu.java
@@ -33,6 +33,7 @@ import org.apache.kudu.client.SessionConfiguration;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 
 import org.apache.nifi.processor.AbstractProcessor;
@@ -61,7 +62,7 @@ public abstract class AbstractKudu extends AbstractProcessor {
             .description("List all kudu masters's ip with port (e.g. 7051), comma separated")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     protected static final PropertyDescriptor TABLE_NAME = new PropertyDescriptor.Builder()
@@ -69,7 +70,7 @@ public abstract class AbstractKudu extends AbstractProcessor {
             .description("The name of the Kudu Table to put data into")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
@@ -117,7 +118,7 @@ public abstract class AbstractKudu extends AbstractProcessor {
             .defaultValue("100")
             .required(true)
             .addValidator(StandardValidators.createLongValidator(2, 100000, true))
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     protected static final Relationship REL_SUCCESS = new Relationship.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/YandexTranslate.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/YandexTranslate.java b/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/YandexTranslate.java
index 3477754..5f1ae88 100644
--- a/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/YandexTranslate.java
+++ b/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/YandexTranslate.java
@@ -30,6 +30,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -75,7 +76,7 @@ import java.util.Set;
 })
 @DynamicProperty(name = "The name of an attribute to set that will contain the translated text of the value",
         value = "The value to translate",
-        supportsExpressionLanguage = true,
+        expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
         description = "User-defined properties are used to translate arbitrary text based on attributes.")
 public class YandexTranslate extends AbstractProcessor {
 
@@ -90,7 +91,7 @@ public class YandexTranslate extends AbstractProcessor {
             .description("The language of incoming data")
             .required(true)
             .defaultValue("es")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(new LanguageNameValidator())
             .build();
     public static final PropertyDescriptor TARGET_LANGUAGE = new PropertyDescriptor.Builder()
@@ -98,7 +99,7 @@ public class YandexTranslate extends AbstractProcessor {
             .description("The language to translate the text into")
             .required(true)
             .defaultValue("en")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(new LanguageNameValidator())
             .build();
     public static final PropertyDescriptor TRANSLATE_CONTENT = new PropertyDescriptor.Builder()
@@ -113,7 +114,7 @@ public class YandexTranslate extends AbstractProcessor {
             .description("Specifies the character set of the data to be translated")
             .required(true)
             .defaultValue("UTF-8")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
             .build();
 
@@ -169,7 +170,7 @@ public class YandexTranslate extends AbstractProcessor {
         return new PropertyDescriptor.Builder()
                 .name(propertyDescriptorName)
                 .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
                 .dynamic(true)
                 .build();
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/image/ResizeImage.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/image/ResizeImage.java b/nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/image/ResizeImage.java
index 176561f..40c034b 100644
--- a/nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/image/ResizeImage.java
+++ b/nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/image/ResizeImage.java
@@ -43,6 +43,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -70,14 +71,14 @@ public class ResizeImage extends AbstractProcessor {
         .name("Image Width (in pixels)")
         .description("The desired number of pixels for the image's width")
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
         .build();
     static final PropertyDescriptor IMAGE_HEIGHT = new PropertyDescriptor.Builder()
         .name("Image Height (in pixels)")
         .description("The desired number of pixels for the image's height")
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
         .build();
     static final PropertyDescriptor SCALING_ALGORITHM = new PropertyDescriptor.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java b/nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java
index 388bc73..c89853b 100644
--- a/nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java
+++ b/nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java
@@ -37,6 +37,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -106,7 +107,7 @@ public class ExtractMediaMetadata extends AbstractProcessor {
                     + " added by the processor.")
             .required(false)
             .addValidator(StandardValidators.ATTRIBUTE_KEY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     static final Relationship SUCCESS = new Relationship.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-task/src/main/java/org/apache/nifi/metrics/reporting/reporter/service/GraphiteMetricReporterService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-task/src/main/java/org/apache/nifi/metrics/reporting/reporter/service/GraphiteMetricReporterService.java b/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-task/src/main/java/org/apache/nifi/metrics/reporting/reporter/service/GraphiteMetricReporterService.java
index 55623ce..32c6449 100644
--- a/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-task/src/main/java/org/apache/nifi/metrics/reporting/reporter/service/GraphiteMetricReporterService.java
+++ b/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-task/src/main/java/org/apache/nifi/metrics/reporting/reporter/service/GraphiteMetricReporterService.java
@@ -28,6 +28,7 @@ import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.metrics.reporting.task.MetricsReportingTask;
 import org.apache.nifi.processor.util.StandardValidators;
 
@@ -57,7 +58,7 @@ public class GraphiteMetricReporterService extends AbstractControllerService imp
             .description("The hostname of the carbon listener")
             .required(true)
             .addValidator(StandardValidators.URI_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     /**
@@ -69,7 +70,7 @@ public class GraphiteMetricReporterService extends AbstractControllerService imp
             .description("The port on which carbon listens")
             .required(true)
             .addValidator(StandardValidators.PORT_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     /**
@@ -94,7 +95,7 @@ public class GraphiteMetricReporterService extends AbstractControllerService imp
             .required(true)
             .defaultValue("nifi")
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     /**

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-task/src/main/java/org/apache/nifi/metrics/reporting/task/MetricsReportingTask.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-task/src/main/java/org/apache/nifi/metrics/reporting/task/MetricsReportingTask.java b/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-task/src/main/java/org/apache/nifi/metrics/reporting/task/MetricsReportingTask.java
index 37eb194..dfd6f79 100644
--- a/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-task/src/main/java/org/apache/nifi/metrics/reporting/task/MetricsReportingTask.java
+++ b/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-task/src/main/java/org/apache/nifi/metrics/reporting/task/MetricsReportingTask.java
@@ -25,6 +25,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.status.ProcessGroupStatus;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.metrics.FlowMetricSet;
 import org.apache.nifi.metrics.reporting.reporter.service.MetricReporterService;
 import org.apache.nifi.processor.util.StandardValidators;
@@ -73,7 +74,7 @@ public class MetricsReportingTask extends AbstractReportingTask {
             .description("The id of the process group to report. If not specified, metrics of the root process group" +
                     "are reported.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessor.java b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessor.java
index bd2c0e3..5bef4a8 100644
--- a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessor.java
+++ b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessor.java
@@ -30,6 +30,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.authentication.exception.ProviderCreationException;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -61,7 +62,7 @@ public abstract class AbstractMongoProcessor extends AbstractProcessor {
         .displayName("Mongo URI")
         .description("MongoURI, typically of the form: mongodb://host1[:port1][,host2[:port2],...]")
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .build();
     protected static final PropertyDescriptor DATABASE_NAME = new PropertyDescriptor.Builder()
@@ -69,14 +70,14 @@ public abstract class AbstractMongoProcessor extends AbstractProcessor {
         .displayName("Mongo Database Name")
         .description("The name of the database to use")
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .build();
     protected static final PropertyDescriptor COLLECTION_NAME = new PropertyDescriptor.Builder()
         .name("Mongo Collection Name")
         .description("The name of the collection to use")
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .build();
     public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
@@ -130,7 +131,7 @@ public abstract class AbstractMongoProcessor extends AbstractProcessor {
             .name("mongo-query-attribute")
             .displayName("Query Output Attribute")
             .description("If set, the query will be written to a specified attribute on the output flowfiles.")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
             .required(false)
             .build();
@@ -141,7 +142,7 @@ public abstract class AbstractMongoProcessor extends AbstractProcessor {
             .required(true)
             .defaultValue("UTF-8")
             .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     static List<PropertyDescriptor> descriptors = new ArrayList<>();
@@ -264,7 +265,8 @@ public abstract class AbstractMongoProcessor extends AbstractProcessor {
         return writeConcern;
     }
 
-    protected void writeBatch(String payload, FlowFile parent, ProcessContext context, ProcessSession session, Map extraAttributes, Relationship rel) throws UnsupportedEncodingException {
+    protected void writeBatch(String payload, FlowFile parent, ProcessContext context, ProcessSession session,
+            Map<String, String> extraAttributes, Relationship rel) throws UnsupportedEncodingException {
         String charset = parent != null ? context.getProperty(CHARSET).evaluateAttributeExpressions(parent).getValue()
                 : context.getProperty(CHARSET).evaluateAttributeExpressions().getValue();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/GetMongo.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/GetMongo.java b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/GetMongo.java
index 7f79e9d..fbd50b6 100644
--- a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/GetMongo.java
+++ b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/GetMongo.java
@@ -31,6 +31,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.logging.ComponentLog;
@@ -95,7 +96,7 @@ public class GetMongo extends AbstractMongoProcessor {
                 "the flowfile's body. If this field is left blank and a timer is enabled instead of an incoming connection, " +
                 "that will result in a full collection fetch using a \"{}\" query.")
         .required(false)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .addValidator(DOCUMENT_VALIDATOR)
         .build();
 
@@ -103,21 +104,21 @@ public class GetMongo extends AbstractMongoProcessor {
             .name("Projection")
             .description("The fields to be returned from the documents in the result set; must be a valid BSON document")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(DOCUMENT_VALIDATOR)
             .build();
     static final PropertyDescriptor SORT = new PropertyDescriptor.Builder()
             .name("Sort")
             .description("The fields by which to sort; must be a valid BSON document")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(DOCUMENT_VALIDATOR)
             .build();
     static final PropertyDescriptor LIMIT = new PropertyDescriptor.Builder()
             .name("Limit")
             .description("The maximum number of elements to return")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
             .build();
 
@@ -125,7 +126,7 @@ public class GetMongo extends AbstractMongoProcessor {
             .name("Batch Size")
             .description("The number of elements returned from the server in one batch")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
             .build();
     static final PropertyDescriptor RESULTS_PER_FLOWFILE = new PropertyDescriptor.Builder()
@@ -133,7 +134,7 @@ public class GetMongo extends AbstractMongoProcessor {
             .displayName("Results Per FlowFile")
             .description("How many results to put into a flowfile at once. The whole body will be treated as a JSON array of results.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
             .build();
 
@@ -164,7 +165,7 @@ public class GetMongo extends AbstractMongoProcessor {
             .description("By default, MongoDB's Java driver returns \"extended JSON\". Some of the features of this variant of JSON" +
                     " may cause problems for other JSON parsers that expect only standard JSON types and conventions. This configuration setting " +
                     " controls whether to use extended JSON or provide a clean view that conforms to standard JSON.")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(true)
             .build();
 
@@ -253,7 +254,7 @@ public class GetMongo extends AbstractMongoProcessor {
 
         final ComponentLog logger = getLogger();
 
-        Map attributes = new HashMap();
+        Map<String, String> attributes = new HashMap<String, String>();
         attributes.put(CoreAttributes.MIME_TYPE.key(), "application/json");
 
         final Document query;

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/RunMongoAggregation.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/RunMongoAggregation.java b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/RunMongoAggregation.java
index d5ec667..668d023 100644
--- a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/RunMongoAggregation.java
+++ b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/RunMongoAggregation.java
@@ -30,11 +30,13 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
+import org.bson.Document;
 import org.bson.conversions.Bson;
 import com.fasterxml.jackson.databind.ObjectMapper;
 
@@ -74,7 +76,7 @@ public class RunMongoAggregation extends AbstractMongoProcessor {
 
         ObjectMapper mapper = new ObjectMapper();
         List<Map> values = mapper.readValue(query, List.class);
-        for (Map val : values) {
+        for (Map<?, ?> val : values) {
             result.add(new BasicDBObject(val));
         }
 
@@ -102,7 +104,7 @@ public class RunMongoAggregation extends AbstractMongoProcessor {
     static final PropertyDescriptor QUERY = new PropertyDescriptor.Builder()
             .name("mongo-agg-query")
             .displayName("Query")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .description("The aggregation query to be executed.")
             .required(true)
             .addValidator(AGG_VALIDATOR)
@@ -137,7 +139,7 @@ public class RunMongoAggregation extends AbstractMongoProcessor {
         return propertyDescriptors;
     }
 
-    static String buildBatch(List batch) {
+    static String buildBatch(List<Document> batch) {
         ObjectMapper mapper = new ObjectMapper();
         String retVal;
         try {
@@ -165,27 +167,27 @@ public class RunMongoAggregation extends AbstractMongoProcessor {
         Integer batchSize = context.getProperty(BATCH_SIZE).asInteger();
         Integer resultsPerFlowfile = context.getProperty(RESULTS_PER_FLOWFILE).asInteger();
 
-        Map attrs = new HashMap();
+        Map<String, String> attrs = new HashMap<String, String>();
         if (queryAttr != null && queryAttr.trim().length() > 0) {
             attrs.put(queryAttr, query);
         }
 
-        MongoCollection collection = getCollection(context);
-        MongoCursor iter = null;
+        MongoCollection<Document> collection = getCollection(context);
+        MongoCursor<Document> iter = null;
 
         try {
             List<Bson> aggQuery = buildAggregationQuery(query);
-            AggregateIterable it = collection.aggregate(aggQuery);
+            AggregateIterable<Document> it = collection.aggregate(aggQuery);
             it.batchSize(batchSize != null ? batchSize : 1);
 
             iter = it.iterator();
-            List batch = new ArrayList();
+            List<Document> batch = new ArrayList<Document>();
 
             while (iter.hasNext()) {
                 batch.add(iter.next());
                 if (batch.size() == resultsPerFlowfile) {
                     writeBatch(buildBatch(batch), flowFile, context, session, attrs, REL_RESULTS);
-                    batch = new ArrayList();
+                    batch = new ArrayList<Document>();
                 }
             }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/test/java/org/apache/nifi/processors/mongodb/GetMongoIT.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/test/java/org/apache/nifi/processors/mongodb/GetMongoIT.java b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/test/java/org/apache/nifi/processors/mongodb/GetMongoIT.java
index a079a2b..71bfc2b 100644
--- a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/test/java/org/apache/nifi/processors/mongodb/GetMongoIT.java
+++ b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/test/java/org/apache/nifi/processors/mongodb/GetMongoIT.java
@@ -359,7 +359,6 @@ public class GetMongoIT {
         runner.setIncomingConnection(true);
         runner.setProperty(GetMongo.QUERY, query);
         runner.setProperty(GetMongo.RESULTS_PER_FLOWFILE, "10");
-        runner.setValidateExpressionUsage(true);
         runner.enqueue("test", attributes);
         runner.run(1, true, true);
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/test/java/org/apache/nifi/processors/mongodb/RunMongoAggregationIT.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/test/java/org/apache/nifi/processors/mongodb/RunMongoAggregationIT.java b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/test/java/org/apache/nifi/processors/mongodb/RunMongoAggregationIT.java
index d6c489a..f2ddbca 100644
--- a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/test/java/org/apache/nifi/processors/mongodb/RunMongoAggregationIT.java
+++ b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/test/java/org/apache/nifi/processors/mongodb/RunMongoAggregationIT.java
@@ -59,7 +59,6 @@ public class RunMongoAggregationIT {
         runner.setProperty(AbstractMongoProcessor.DATABASE_NAME, "${db}");
         runner.setProperty(AbstractMongoProcessor.COLLECTION_NAME, "${collection}");
         runner.setProperty(RunMongoAggregation.QUERY_ATTRIBUTE, AGG_ATTR);
-        runner.setValidateExpressionUsage(true);
 
         mongoClient = new MongoClient(new MongoClientURI(MONGO_URI));
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-services/src/main/java/org/apache/nifi/mongodb/AbstractMongoDBControllerService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-services/src/main/java/org/apache/nifi/mongodb/AbstractMongoDBControllerService.java b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-services/src/main/java/org/apache/nifi/mongodb/AbstractMongoDBControllerService.java
index 8ac05b2..5b6c97e 100644
--- a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-services/src/main/java/org/apache/nifi/mongodb/AbstractMongoDBControllerService.java
+++ b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-services/src/main/java/org/apache/nifi/mongodb/AbstractMongoDBControllerService.java
@@ -30,6 +30,7 @@ import org.apache.nifi.authentication.exception.ProviderCreationException;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.security.util.SslContextFactory;
@@ -54,7 +55,7 @@ public class AbstractMongoDBControllerService extends AbstractControllerService
             .displayName("Mongo URI")
             .description("MongoURI, typically of the form: mongodb://host1[:port1][,host2[:port2],...]")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(Validation.DOCUMENT_VALIDATOR)
             .build();
     protected static final PropertyDescriptor DATABASE_NAME = new PropertyDescriptor.Builder()
@@ -62,7 +63,7 @@ public class AbstractMongoDBControllerService extends AbstractControllerService
             .displayName("Mongo Database Name")
             .description("The name of the database to use")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
     protected static final PropertyDescriptor COLLECTION_NAME = new PropertyDescriptor.Builder()
@@ -70,7 +71,7 @@ public class AbstractMongoDBControllerService extends AbstractControllerService
             .displayName("Mongo Collection Name")
             .description("The name of the collection to use")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
     public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()


[09/15] nifi git commit: NIFI-4149 - Indicate if EL is evaluated against FFs or not - take into account input requirement for documentation rendering - Renamed variable registry scope and added comments - Doc + change in mock framework to check scope + u

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
index 119bbfd..d0bd539 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
@@ -103,6 +103,7 @@ import org.apache.nifi.controller.status.history.GarbageCollectionStatus;
 import org.apache.nifi.diagnostics.GarbageCollection;
 import org.apache.nifi.diagnostics.StorageUsage;
 import org.apache.nifi.diagnostics.SystemDiagnostics;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.groups.ProcessGroup;
@@ -3600,6 +3601,12 @@ public final class DtoFactory {
         dto.setDefaultValue(propertyDescriptor.getDefaultValue());
         dto.setSupportsEl(propertyDescriptor.isExpressionLanguageSupported());
 
+        // to support legacy/deprecated method .expressionLanguageSupported(true)
+        String description = propertyDescriptor.isExpressionLanguageSupported()
+                && propertyDescriptor.getExpressionLanguageScope().equals(ExpressionLanguageScope.NONE)
+                ? "true (undefined scope)" : propertyDescriptor.getExpressionLanguageScope().getDescription();
+        dto.setScopeEl(description);
+
         // set the identifies controller service is applicable
         if (propertyDescriptor.getControllerServiceDefinition() != null) {
             final Class serviceClass = propertyDescriptor.getControllerServiceDefinition();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js
index 42f0deb..238c8be 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js
@@ -900,7 +900,7 @@
                     tipContent.push('<b>Default value:</b> ' + nfCommon.escapeHtml(propertyDescriptor.defaultValue));
                 }
                 if (!nfCommon.isBlank(propertyDescriptor.supportsEl)) {
-                    tipContent.push('<b>Supports expression language:</b> ' + nfCommon.escapeHtml(propertyDescriptor.supportsEl));
+                    tipContent.push('<b>Expression language scope:</b> ' + nfCommon.escapeHtml(propertyDescriptor.scopeEl));
                 }
                 if (!nfCommon.isBlank(propertyDescriptor.identifiesControllerService)) {
                     var formattedType = nfCommon.formatType({

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/AbstractGCPProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/AbstractGCPProcessor.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/AbstractGCPProcessor.java
index 9b9a253..5f51194 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/AbstractGCPProcessor.java
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/AbstractGCPProcessor.java
@@ -22,6 +22,7 @@ import com.google.cloud.Service;
 import com.google.common.collect.ImmutableList;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.util.StandardValidators;
@@ -60,7 +61,7 @@ public abstract class AbstractGCPProcessor<
             .displayName("Proxy host")
             .description("IP or hostname of the proxy to be used")
             .required(false)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -69,7 +70,7 @@ public abstract class AbstractGCPProcessor<
             .displayName("Proxy port")
             .description("Proxy port number")
             .required(false)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .addValidator(StandardValidators.INTEGER_VALIDATOR)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialPropertyDescriptors.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialPropertyDescriptors.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialPropertyDescriptors.java
index bd99dd8..0728eb0 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialPropertyDescriptors.java
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialPropertyDescriptors.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.processors.gcp.credentials.factory;
 
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.util.StandardValidators;
 
 /**
@@ -36,7 +37,7 @@ public final class CredentialPropertyDescriptors {
     public static final PropertyDescriptor USE_APPLICATION_DEFAULT_CREDENTIALS = new PropertyDescriptor.Builder()
             .name("application-default-credentials")
             .displayName("Use Application Default Credentials")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(false)
             .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
             .sensitive(false)
@@ -51,7 +52,7 @@ public final class CredentialPropertyDescriptors {
     public static final PropertyDescriptor USE_COMPUTE_ENGINE_CREDENTIALS = new PropertyDescriptor.Builder()
             .name("compute-engine-credentials")
             .displayName("Use Compute Engine Credentials")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(false)
             .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
             .sensitive(false)
@@ -71,7 +72,7 @@ public final class CredentialPropertyDescriptors {
     public static final PropertyDescriptor SERVICE_ACCOUNT_JSON_FILE = new PropertyDescriptor.Builder()
             .name("service-account-json-file")
             .displayName("Service Account JSON File")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(false)
             .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
             .description("Path to a file containing a Service Account key file in JSON format.")
@@ -80,7 +81,7 @@ public final class CredentialPropertyDescriptors {
     public static final PropertyDescriptor SERVICE_ACCOUNT_JSON = new PropertyDescriptor.Builder()
             .name("service-account-json")
             .displayName("Service Account JSON")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .description("The raw JSON containing a Service Account keyfile.")

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/DeleteGCSObject.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/DeleteGCSObject.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/DeleteGCSObject.java
index 80bb9c0..66d51db 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/DeleteGCSObject.java
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/DeleteGCSObject.java
@@ -25,6 +25,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -52,7 +53,7 @@ public class DeleteGCSObject extends AbstractGCSProcessor {
             .description(BUCKET_DESC)
             .required(true)
             .defaultValue("${" + BUCKET_ATTR + "}")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -62,7 +63,7 @@ public class DeleteGCSObject extends AbstractGCSProcessor {
             .description(KEY_DESC)
             .required(true)
             .defaultValue("${filename}")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -71,7 +72,7 @@ public class DeleteGCSObject extends AbstractGCSProcessor {
             .displayName("Generation")
             .description("The generation of the object to be deleted. If null, will use latest version of the object.")
             .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(false)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/FetchGCSObject.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/FetchGCSObject.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/FetchGCSObject.java
index a65158a..2adfd29 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/FetchGCSObject.java
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/FetchGCSObject.java
@@ -32,6 +32,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.ProcessContext;
@@ -129,7 +130,7 @@ public class FetchGCSObject extends AbstractGCSProcessor {
             .description(BUCKET_DESC)
             .required(true)
             .defaultValue("${" + BUCKET_ATTR + "}")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -139,7 +140,7 @@ public class FetchGCSObject extends AbstractGCSProcessor {
             .description(KEY_DESC)
             .required(true)
             .defaultValue("${" + CoreAttributes.FILENAME.key() + "}")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -147,7 +148,7 @@ public class FetchGCSObject extends AbstractGCSProcessor {
             .name("gcs-generation")
             .displayName("Object Generation")
             .description("The generation of the Object to download. If null, will download latest generation.")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
             .required(false)
             .build();
@@ -157,7 +158,7 @@ public class FetchGCSObject extends AbstractGCSProcessor {
             .displayName("Server Side Encryption Key")
             .description("An AES256 Key (encoded in base64) which the object has been encrypted in.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .sensitive(true)
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/ListGCSBucket.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/ListGCSBucket.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/ListGCSBucket.java
index 4af4ae2..6bb3812 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/ListGCSBucket.java
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/ListGCSBucket.java
@@ -35,6 +35,7 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.ProcessContext;
@@ -147,7 +148,7 @@ public class ListGCSBucket extends AbstractGCSProcessor {
             .displayName("Bucket")
             .description(BUCKET_DESC)
             .required(true)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -162,7 +163,7 @@ public class ListGCSBucket extends AbstractGCSProcessor {
     public static final PropertyDescriptor USE_GENERATIONS = new PropertyDescriptor.Builder()
             .name("gcs-use-generations")
             .displayName("Use Generations")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(true)
             .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
             .allowableValues("true", "false")

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/PutGCSObject.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/PutGCSObject.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/PutGCSObject.java
index 2b77b1b..5be6e39 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/PutGCSObject.java
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/PutGCSObject.java
@@ -34,6 +34,7 @@ import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.ProcessContext;
@@ -111,7 +112,7 @@ import static org.apache.nifi.processors.gcp.storage.StorageAttributes.URI_DESC;
 @DynamicProperty(name = "The name of a User-Defined Metadata field to add to the GCS Object",
         value = "The value of a User-Defined Metadata field to add to the GCS Object",
         description = "Allows user-defined metadata to be added to the GCS object as key/value pairs",
-        supportsExpressionLanguage = true)
+        expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
 @ReadsAttributes({
         @ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the " +
                 "GCS object"),
@@ -150,7 +151,7 @@ public class PutGCSObject extends AbstractGCSProcessor {
             .description(BUCKET_DESC)
             .required(true)
             .defaultValue("${" + BUCKET_ATTR + "}")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -160,7 +161,7 @@ public class PutGCSObject extends AbstractGCSProcessor {
             .description(KEY_DESC)
             .required(true)
             .defaultValue("${filename}")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -170,7 +171,7 @@ public class PutGCSObject extends AbstractGCSProcessor {
                       .description("Content Type for the file, i.e. text/plain")
                       .defaultValue("${mime.type}")
                       .required(false)
-                      .expressionLanguageSupported(true)
+                      .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
                       .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
                       .build();
 
@@ -179,7 +180,7 @@ public class PutGCSObject extends AbstractGCSProcessor {
             .displayName("MD5 Hash")
             .description("MD5 Hash (encoded in Base64) of the file for server-side validation.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -189,7 +190,7 @@ public class PutGCSObject extends AbstractGCSProcessor {
             .displayName("CRC32C Checksum")
             .description("CRC32C Checksum (encoded in Base64, big-Endian order) of the file for server-side validation.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -252,7 +253,7 @@ public class PutGCSObject extends AbstractGCSProcessor {
             .displayName("Server Side Encryption Key")
             .description("An AES256 Encryption Key (encoded in base64) for server-side encryption of the object.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .sensitive(true)
             .build();
@@ -307,7 +308,7 @@ public class PutGCSObject extends AbstractGCSProcessor {
         return new PropertyDescriptor.Builder()
                 .name(propertyDescriptorName)
                 .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
                 .dynamic(true)
                 .build();
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/src/main/java/org/apache/nifi/processors/groovyx/ExecuteGroovyScript.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/src/main/java/org/apache/nifi/processors/groovyx/ExecuteGroovyScript.java b/nifi-nar-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/src/main/java/org/apache/nifi/processors/groovyx/ExecuteGroovyScript.java
index 30c5280..ce89bdd 100644
--- a/nifi-nar-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/src/main/java/org/apache/nifi/processors/groovyx/ExecuteGroovyScript.java
+++ b/nifi-nar-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/src/main/java/org/apache/nifi/processors/groovyx/ExecuteGroovyScript.java
@@ -34,6 +34,7 @@ import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.dbcp.DBCPService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -78,7 +79,7 @@ import java.util.Set;
 @SeeAlso(classNames={"org.apache.nifi.processors.script.ExecuteScript"})
 @DynamicProperty(name = "A script engine property to update",
         value = "The value to set it to",
-        supportsExpressionLanguage = true,
+        expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
         description = "Updates a script engine property specified by the Dynamic Property's key with the value "
                 + "specified by the Dynamic Property's value. Use `CTL.` to access any controller services.")
 public class ExecuteGroovyScript extends AbstractProcessor {
@@ -94,7 +95,7 @@ public class ExecuteGroovyScript extends AbstractProcessor {
             .required(false)
             .description("Path to script file to execute. Only one of Script File or Script Body may be used")
             .addValidator(Validators.createFileExistsAndReadableValidator())
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor SCRIPT_BODY = new PropertyDescriptor.Builder()
@@ -103,7 +104,7 @@ public class ExecuteGroovyScript extends AbstractProcessor {
             .required(false)
             .description("Body of script to execute. Only one of Script File or Script Body may be used")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
 
     public static String[] VALID_FAIL_STRATEGY = {"rollback", "transfer to failure"};
@@ -116,14 +117,19 @@ public class ExecuteGroovyScript extends AbstractProcessor {
                     +" If `rollback` selected and unhandled exception occurred then all flowFiles received from incoming queues will be penalized and returned."
                     +" If the processor has no incoming connections then this parameter has no effect."
                 )
-            .required(true).expressionLanguageSupported(false).allowableValues(VALID_FAIL_STRATEGY).defaultValue(VALID_FAIL_STRATEGY[0]).build();
+            .required(true).expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues(VALID_FAIL_STRATEGY)
+            .defaultValue(VALID_FAIL_STRATEGY[0])
+            .build();
 
     public static final PropertyDescriptor ADD_CLASSPATH = new PropertyDescriptor.Builder()
             .name("groovyx-additional-classpath")
             .displayName("Additional classpath")
             .required(false)
             .description("Classpath list separated by semicolon. You can use masks like `*`, `*.jar` in file name.")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR).expressionLanguageSupported(true).build();
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
 
     public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success").description("FlowFiles that were successfully processed").build();
 
@@ -491,7 +497,7 @@ public class ExecuteGroovyScript extends AbstractProcessor {
                 .name(propertyDescriptorName)
                 .required(false)
                 .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
                 .dynamic(true)
                 .build();
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/DeleteHDFS.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/DeleteHDFS.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/DeleteHDFS.java
index 1c09e79..171fbd1 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/DeleteHDFS.java
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/DeleteHDFS.java
@@ -32,6 +32,7 @@ import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -90,7 +91,7 @@ public class DeleteHDFS extends AbstractHadoopProcessor {
             .description("The HDFS file or directory to delete. A wildcard expression may be used to only delete certain files")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor RECURSIVE = new PropertyDescriptor.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
index 8929851..f037e33 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
@@ -36,6 +36,7 @@ import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.ProcessContext;
@@ -74,7 +75,7 @@ public class FetchHDFS extends AbstractHadoopProcessor {
         .name("HDFS Filename")
         .description("The name of the HDFS file to retrieve")
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .defaultValue("${path}/${filename}")
         .addValidator(StandardValidators.ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR)
         .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/MoveHDFS.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/MoveHDFS.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/MoveHDFS.java
index 9af0f93..ea61ed1 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/MoveHDFS.java
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/MoveHDFS.java
@@ -36,6 +36,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.ProcessContext;
@@ -98,19 +99,23 @@ public class MoveHDFS extends AbstractHadoopProcessor {
     public static final String ABSOLUTE_HDFS_PATH_ATTRIBUTE = "absolute.hdfs.path";
 
     // relationships
-    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
             .description("Files that have been successfully renamed on HDFS are transferred to this relationship")
             .build();
 
-    public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure")
-            .description("Files that could not be renamed on HDFS are transferred to this relationship").build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Files that could not be renamed on HDFS are transferred to this relationship")
+            .build();
 
     // properties
     public static final PropertyDescriptor CONFLICT_RESOLUTION = new PropertyDescriptor.Builder()
             .name("Conflict Resolution Strategy")
             .description(
                     "Indicates what should happen when a file with the same name already exists in the output directory")
-            .required(true).defaultValue(FAIL_RESOLUTION_AV.getValue())
+            .required(true)
+            .defaultValue(FAIL_RESOLUTION_AV.getValue())
             .allowableValues(REPLACE_RESOLUTION_AV, IGNORE_RESOLUTION_AV, FAIL_RESOLUTION_AV).build();
 
     public static final PropertyDescriptor FILE_FILTER_REGEX = new PropertyDescriptor.Builder()
@@ -118,37 +123,55 @@ public class MoveHDFS extends AbstractHadoopProcessor {
             .description(
                     "A Java Regular Expression for filtering Filenames; if a filter is supplied then only files whose names match that Regular "
                             + "Expression will be fetched, otherwise all files will be fetched")
-            .required(false).addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR).build();
+            .required(false)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .build();
 
     public static final PropertyDescriptor IGNORE_DOTTED_FILES = new PropertyDescriptor.Builder()
             .name("Ignore Dotted Files")
-            .description("If true, files whose names begin with a dot (\".\") will be ignored").required(true)
-            .allowableValues("true", "false").defaultValue("true").build();
+            .description("If true, files whose names begin with a dot (\".\") will be ignored")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
 
     public static final PropertyDescriptor INPUT_DIRECTORY_OR_FILE = new PropertyDescriptor.Builder()
             .name("Input Directory or File")
             .description("The HDFS directory from which files should be read, or a single file to read.")
-            .defaultValue("${path}").addValidator(StandardValidators.ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR)
-            .expressionLanguageSupported(true).build();
+            .defaultValue("${path}")
+            .addValidator(StandardValidators.ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
 
-    public static final PropertyDescriptor OUTPUT_DIRECTORY = new PropertyDescriptor.Builder().name("Output Directory")
-            .description("The HDFS directory where the files will be moved to").required(true)
-            .addValidator(StandardValidators.ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR).expressionLanguageSupported(true)
+    public static final PropertyDescriptor OUTPUT_DIRECTORY = new PropertyDescriptor.Builder()
+            .name("Output Directory")
+            .description("The HDFS directory where the files will be moved to")
+            .required(true)
+            .addValidator(StandardValidators.ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
-    public static final PropertyDescriptor OPERATION = new PropertyDescriptor.Builder().name("HDFS Operation")
-            .description("The operation that will be performed on the source file").required(true)
-            .allowableValues("move", "copy").defaultValue("move").build();
+    public static final PropertyDescriptor OPERATION = new PropertyDescriptor.Builder()
+            .name("HDFS Operation")
+            .description("The operation that will be performed on the source file")
+            .required(true)
+            .allowableValues("move", "copy")
+            .defaultValue("move")
+            .build();
 
-    public static final PropertyDescriptor REMOTE_OWNER = new PropertyDescriptor.Builder().name("Remote Owner")
+    public static final PropertyDescriptor REMOTE_OWNER = new PropertyDescriptor.Builder()
+            .name("Remote Owner")
             .description(
                     "Changes the owner of the HDFS file to this value after it is written. This only works if NiFi is running as a user that has HDFS super user privilege to change owner")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
 
-    public static final PropertyDescriptor REMOTE_GROUP = new PropertyDescriptor.Builder().name("Remote Group")
+    public static final PropertyDescriptor REMOTE_GROUP = new PropertyDescriptor.Builder()
+            .name("Remote Group")
             .description(
                     "Changes the group of the HDFS file to this value after it is written. This only works if NiFi is running as a user that has HDFS super user privilege to change group")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
 
     static {
         final Set<Relationship> rels = new HashSet<>();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
index 8c8bf16..91fd204 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
@@ -37,6 +37,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.DataUnit;
@@ -150,7 +151,7 @@ public class PutHDFS extends AbstractHadoopProcessor {
             .description(
                     "Changes the owner of the HDFS file to this value after it is written. This only works if NiFi is running as a user that has HDFS super user privilege to change owner")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor REMOTE_GROUP = new PropertyDescriptor.Builder()
@@ -158,7 +159,7 @@ public class PutHDFS extends AbstractHadoopProcessor {
             .description(
                     "Changes the group of the HDFS file to this value after it is written. This only works if NiFi is running as a user that has HDFS super user privilege to change group")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     private static final Set<Relationship> relationships;

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/inotify/GetHDFSEvents.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/inotify/GetHDFSEvents.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/inotify/GetHDFSEvents.java
index 234d1ff..ee43e70 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/inotify/GetHDFSEvents.java
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/inotify/GetHDFSEvents.java
@@ -37,6 +37,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.components.state.StateManager;
 import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.ProcessContext;
@@ -100,7 +101,7 @@ public class GetHDFSEvents extends AbstractHadoopProcessor {
             .description("The HDFS path to get event notifications for. This property accepts both expression language and regular expressions. This will be evaluated during the " +
                     "OnScheduled phase.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.createRegexValidator(0, Integer.MAX_VALUE, true))
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/AbstractDeleteHBase.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/AbstractDeleteHBase.java b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/AbstractDeleteHBase.java
index a097fbe..43df2c8 100644
--- a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/AbstractDeleteHBase.java
+++ b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/AbstractDeleteHBase.java
@@ -19,6 +19,7 @@ package org.apache.nifi.hbase;
 
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -42,14 +43,14 @@ public abstract class AbstractDeleteHBase extends AbstractProcessor {
             .name("Table Name")
             .description("The name of the HBase Table.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
     protected static final PropertyDescriptor ROW_ID = new PropertyDescriptor.Builder()
             .name("Row Identifier")
             .description("Specifies the Row ID to use when deleting data into HBase")
             .required(false) // not all sub-classes will require this
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/AbstractPutHBase.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/AbstractPutHBase.java b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/AbstractPutHBase.java
index ac24baa..db12936 100644
--- a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/AbstractPutHBase.java
+++ b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/AbstractPutHBase.java
@@ -28,6 +28,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.hbase.put.PutFlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -52,14 +53,14 @@ public abstract class AbstractPutHBase extends AbstractProcessor {
             .name("Table Name")
             .description("The name of the HBase Table to put data into")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
     protected static final PropertyDescriptor ROW_ID = new PropertyDescriptor.Builder()
             .name("Row Identifier")
             .description("Specifies the Row ID to use when inserting data into HBase")
             .required(false) // not all sub-classes will require this
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -80,7 +81,7 @@ public abstract class AbstractPutHBase extends AbstractProcessor {
                     " to the correct byte[] representation. The Binary option should be used if you are using Binary row" +
                     " keys in HBase")
             .required(false) // not all sub-classes will require this
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .defaultValue(ROW_ID_ENCODING_STRING.getValue())
             .allowableValues(ROW_ID_ENCODING_STRING,ROW_ID_ENCODING_BINARY)
             .build();
@@ -88,21 +89,21 @@ public abstract class AbstractPutHBase extends AbstractProcessor {
             .name("Column Family")
             .description("The Column Family to use when inserting data into HBase")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
     protected static final PropertyDescriptor COLUMN_QUALIFIER = new PropertyDescriptor.Builder()
             .name("Column Qualifier")
             .description("The Column Qualifier to use when inserting data into HBase")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
     protected static final PropertyDescriptor TIMESTAMP = new PropertyDescriptor.Builder()
             .name("timestamp")
             .displayName("Timestamp")
             .description("The timestamp for the cells being created in HBase. This field can be left blank and HBase will use the current time.")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
             .build();
     protected static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/DeleteHBaseRow.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/DeleteHBaseRow.java b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/DeleteHBaseRow.java
index 8aec55a..9fe3882 100644
--- a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/DeleteHBaseRow.java
+++ b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/DeleteHBaseRow.java
@@ -25,6 +25,7 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -72,7 +73,7 @@ public class DeleteHBaseRow extends AbstractDeleteHBase {
             .required(true)
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
             .defaultValue("5")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
 
     static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
@@ -82,7 +83,7 @@ public class DeleteHBaseRow extends AbstractDeleteHBase {
             .required(true)
             .defaultValue("50")
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
 
     static final PropertyDescriptor KEY_SEPARATOR = new PropertyDescriptor.Builder()
@@ -93,7 +94,7 @@ public class DeleteHBaseRow extends AbstractDeleteHBase {
             .required(true)
             .defaultValue(",")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     static final PropertyDescriptor CHARSET = new PropertyDescriptor.Builder()
             .name("delete-char-set")

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/FetchHBaseRow.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/FetchHBaseRow.java b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/FetchHBaseRow.java
index 654f7af..d8e4a9a 100644
--- a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/FetchHBaseRow.java
+++ b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/FetchHBaseRow.java
@@ -25,6 +25,7 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.hbase.io.JsonFullRowSerializer;
@@ -77,7 +78,7 @@ public class FetchHBaseRow extends AbstractProcessor {
             .name("Table Name")
             .description("The name of the HBase Table to fetch from.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -85,7 +86,7 @@ public class FetchHBaseRow extends AbstractProcessor {
             .name("Row Identifier")
             .description("The identifier of the row to fetch.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -94,7 +95,7 @@ public class FetchHBaseRow extends AbstractProcessor {
             .description("An optional comma-separated list of \"<colFamily>:<colQualifier>\" pairs to fetch. To return all columns " +
                     "for a given family, leave off the qualifier such as \"<colFamily1>,<colFamily2>\".")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.createRegexMatchingValidator(COLUMNS_PATTERN))
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/GetHBase.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/GetHBase.java b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/GetHBase.java
index dee37c6..8170bc6 100644
--- a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/GetHBase.java
+++ b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/GetHBase.java
@@ -59,6 +59,7 @@ import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.components.state.StateManager;
 import org.apache.nifi.components.state.StateMap;
 import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.hbase.io.JsonRowSerializer;
 import org.apache.nifi.hbase.io.RowSerializer;
@@ -121,7 +122,7 @@ public class GetHBase extends AbstractProcessor {
             .name("Table Name")
             .description("The name of the HBase Table to put data into")
             .required(true)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
     static final PropertyDescriptor COLUMNS = new PropertyDescriptor.Builder()
@@ -129,14 +130,14 @@ public class GetHBase extends AbstractProcessor {
             .description("A comma-separated list of \"<colFamily>:<colQualifier>\" pairs to return when scanning. To return all columns " +
                     "for a given family, leave off the qualifier such as \"<colFamily1>,<colFamily2>\".")
             .required(false)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .addValidator(StandardValidators.createRegexMatchingValidator(COLUMNS_PATTERN))
             .build();
     static final PropertyDescriptor FILTER_EXPRESSION = new PropertyDescriptor.Builder()
             .name("Filter Expression")
             .description("An HBase filter expression that will be applied to the scan. This property can not be used when also using the Columns property.")
             .required(false)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
     static final PropertyDescriptor INITIAL_TIMERANGE = new PropertyDescriptor.Builder()
@@ -144,7 +145,7 @@ public class GetHBase extends AbstractProcessor {
             .description("The time range to use on the first scan of a table. None will pull the entire table on the first scan, " +
                     "Current Time will pull entries from that point forward.")
             .required(true)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .allowableValues(NONE, CURRENT_TIME)
             .defaultValue(NONE.getValue())
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseJSON.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseJSON.java b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseJSON.java
index dc1766a..bee188a 100644
--- a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseJSON.java
+++ b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseJSON.java
@@ -29,6 +29,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.hbase.put.PutColumn;
 import org.apache.nifi.hbase.put.PutFlowFile;
@@ -67,7 +68,7 @@ public class PutHBaseJSON extends AbstractPutHBase {
     protected static final PropertyDescriptor ROW_FIELD_NAME = new PropertyDescriptor.Builder()
             .name("Row Identifier Field Name")
             .description("Specifies the name of a JSON element whose value should be used as the row id for the given JSON document.")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -84,7 +85,7 @@ public class PutHBaseJSON extends AbstractPutHBase {
     protected static final PropertyDescriptor COMPLEX_FIELD_STRATEGY = new PropertyDescriptor.Builder()
             .name("Complex Field Strategy")
             .description("Indicates how to handle complex fields, i.e. fields that do not have a single text value.")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(true)
             .allowableValues(COMPLEX_FIELD_FAIL, COMPLEX_FIELD_WARN, COMPLEX_FIELD_IGNORE, COMPLEX_FIELD_TEXT)
             .defaultValue(COMPLEX_FIELD_TEXT.getValue())

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseRecord.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseRecord.java b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseRecord.java
index b4de3c6..30701c7 100755
--- a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseRecord.java
+++ b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseRecord.java
@@ -25,6 +25,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.hbase.put.PutColumn;
 import org.apache.nifi.hbase.put.PutFlowFile;
@@ -61,7 +62,7 @@ public class PutHBaseRecord extends AbstractPutHBase {
             .name("Row Identifier Field Name")
             .description("Specifies the name of a record field whose value should be used as the row id for the given record.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -71,7 +72,7 @@ public class PutHBaseRecord extends AbstractPutHBase {
             .description("Specifies the name of a record field whose value should be used as the timestamp for the cells in HBase. " +
                     "The value of this field must be a number, string, or date that can be converted to a long. " +
                     "If this field is left blank, HBase will use the current time.")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -96,7 +97,7 @@ public class PutHBaseRecord extends AbstractPutHBase {
     protected static final PropertyDescriptor COMPLEX_FIELD_STRATEGY = new PropertyDescriptor.Builder()
             .name("Complex Field Strategy")
             .description("Indicates how to handle complex fields, i.e. fields that do not have a single text value.")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(true)
             .allowableValues(COMPLEX_FIELD_FAIL, COMPLEX_FIELD_WARN, COMPLEX_FIELD_IGNORE, COMPLEX_FIELD_TEXT)
             .defaultValue(COMPLEX_FIELD_TEXT.getValue())

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/ScanHBase.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/ScanHBase.java b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/ScanHBase.java
index f8782e3..4abd470 100644
--- a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/ScanHBase.java
+++ b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/ScanHBase.java
@@ -39,6 +39,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.hbase.io.JsonFullRowSerializer;
@@ -86,7 +87,7 @@ public class ScanHBase extends AbstractProcessor {
             .name("scanhbase-table-name")
             .description("The name of the HBase Table to fetch from.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -95,7 +96,7 @@ public class ScanHBase extends AbstractProcessor {
             .name("scanhbase-start-rowkey")
             .description("The rowkey to start scan from.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -104,7 +105,7 @@ public class ScanHBase extends AbstractProcessor {
             .name("scanhbase-end-rowkey")
             .description("The row key to end scan by.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -113,7 +114,7 @@ public class ScanHBase extends AbstractProcessor {
             .name("scanhbase-time-range-min")
             .description("Time range min value. Both min and max values for time range should be either blank or provided.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.LONG_VALIDATOR)
             .build();
 
@@ -122,7 +123,7 @@ public class ScanHBase extends AbstractProcessor {
             .name("scanhbase-time-range-max")
             .description("Time range max value. Both min and max values for time range should be either blank or provided.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.LONG_VALIDATOR)
             .build();
 
@@ -131,7 +132,7 @@ public class ScanHBase extends AbstractProcessor {
             .name("scanhbase-limit")
             .description("Limit number of rows retrieved by scan.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.INTEGER_VALIDATOR)
             .build();
 
@@ -140,7 +141,7 @@ public class ScanHBase extends AbstractProcessor {
             .name("scanhbase-bulk-size")
             .description("Limits number of rows in single flow file content. Set to 0 to avoid multiple flow files.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("0")
             .addValidator(StandardValidators.INTEGER_VALIDATOR)
             .build();
@@ -149,7 +150,7 @@ public class ScanHBase extends AbstractProcessor {
             .displayName("Reversed order")
             .name("scanhbase-reversed-order")
             .description("Set whether this scan is a reversed one. This is false by default which means forward(normal) scan.")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .allowableValues("true", "false")
             .required(false)
             .defaultValue("false")
@@ -162,7 +163,7 @@ public class ScanHBase extends AbstractProcessor {
             .description("An HBase filter expression that will be applied to the scan. This property can not be used when also using the Columns property. "
                     + "Example: \"ValueFilter( =, 'binaryprefix:commit' )\"")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -172,7 +173,7 @@ public class ScanHBase extends AbstractProcessor {
             .description("An optional comma-separated list of \"<colFamily>:<colQualifier>\" pairs to fetch. To return all columns " +
                     "for a given family, leave off the qualifier such as \"<colFamily1>,<colFamily2>\".")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.createRegexMatchingValidator(COLUMNS_PATTERN))
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/test/java/org/apache/nifi/hbase/TestDeleteHBaseRow.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/test/java/org/apache/nifi/hbase/TestDeleteHBaseRow.java b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/test/java/org/apache/nifi/hbase/TestDeleteHBaseRow.java
index 6c0d92b..fe819dd 100644
--- a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/test/java/org/apache/nifi/hbase/TestDeleteHBaseRow.java
+++ b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/test/java/org/apache/nifi/hbase/TestDeleteHBaseRow.java
@@ -109,7 +109,6 @@ public class TestDeleteHBaseRow {
 
     @Test
     public void testDeleteWithELSeparator() {
-        runner.setValidateExpressionUsage(true);
         Map<String, String> attrs = new HashMap<>();
         attrs.put("test.separator", "____");
         testSeparatedDeletes("${test.separator}", "____", attrs);
@@ -129,7 +128,6 @@ public class TestDeleteHBaseRow {
         runner.setProperty(DeleteHBaseRow.ROW_ID, "${part_0}-${part_1}-${part_2}-${part_3}-${part_4}");
         runner.setProperty(DeleteHBaseRow.ROW_ID_LOCATION, DeleteHBaseRow.ROW_ID_ATTR);
         runner.setProperty(DeleteHBaseRow.BATCH_SIZE, "200");
-        runner.setValidateExpressionUsage(true);
         runner.run(1, true);
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/dbcp/hive/HiveConnectionPool.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/dbcp/hive/HiveConnectionPool.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/dbcp/hive/HiveConnectionPool.java
index 3972d4e..648cd42 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/dbcp/hive/HiveConnectionPool.java
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/dbcp/hive/HiveConnectionPool.java
@@ -57,6 +57,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.nifi.controller.ControllerServiceInitializationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 
 /**
  * Implementation for Database Connection Pooling Service used for Apache Hive
@@ -77,7 +78,7 @@ public class HiveConnectionPool extends AbstractControllerService implements Hiv
             .defaultValue(null)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor HIVE_CONFIGURATION_RESOURCES = new PropertyDescriptor.Builder()
@@ -88,7 +89,7 @@ public class HiveConnectionPool extends AbstractControllerService implements Hiv
                     + "with Kerberos e.g., the appropriate properties must be set in the configuration files. Please see the Hive documentation for more details.")
             .required(false)
             .addValidator(HiveUtils.createMultipleFilesExistValidator())
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor DB_USER = new PropertyDescriptor.Builder()
@@ -97,7 +98,7 @@ public class HiveConnectionPool extends AbstractControllerService implements Hiv
             .description("Database user name")
             .defaultValue(null)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor DB_PASSWORD = new PropertyDescriptor.Builder()
@@ -108,7 +109,7 @@ public class HiveConnectionPool extends AbstractControllerService implements Hiv
             .required(false)
             .sensitive(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
@@ -119,7 +120,7 @@ public class HiveConnectionPool extends AbstractControllerService implements Hiv
             .defaultValue("500 millis")
             .required(true)
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
@@ -130,7 +131,7 @@ public class HiveConnectionPool extends AbstractControllerService implements Hiv
             .defaultValue("8")
             .required(true)
             .addValidator(StandardValidators.INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor VALIDATION_QUERY = new PropertyDescriptor.Builder()
@@ -141,7 +142,7 @@ public class HiveConnectionPool extends AbstractControllerService implements Hiv
                     + "NOTE: Using validation may have a performance penalty.")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     static final PropertyDescriptor KERBEROS_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ConvertAvroToORC.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ConvertAvroToORC.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ConvertAvroToORC.java
index 073eb6c..f211ac5 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ConvertAvroToORC.java
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ConvertAvroToORC.java
@@ -34,6 +34,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -133,7 +134,7 @@ public class ConvertAvroToORC extends AbstractProcessor {
                     + "If this property is not provided, the full name (including namespace) of the incoming Avro record will be normalized "
                     + "and used as the table name.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveQL.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveQL.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveQL.java
index c68bce8..d066967 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveQL.java
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveQL.java
@@ -29,6 +29,7 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.dbcp.hive.HiveDBCPService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -95,7 +96,7 @@ public class PutHiveQL extends AbstractHiveQLProcessor {
             .required(true)
             .defaultValue(";")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
 
     public static final Relationship REL_SUCCESS = new Relationship.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveStreaming.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveStreaming.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveStreaming.java
index 48925b5..40ba9e2 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveStreaming.java
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveStreaming.java
@@ -42,6 +42,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.hadoop.KerberosProperties;
 import org.apache.nifi.hadoop.SecurityUtil;
@@ -166,7 +167,7 @@ public class PutHiveStreaming extends AbstractSessionFactoryProcessor {
             .description("The URI location for the Hive Metastore. Note that this is not the location of the Hive Server. The default port for the "
                     + "Hive metastore is 9043.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.URI_VALIDATOR)
             .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("(^[^/]+.*[^/]+$|^[^/]+$|^$)"))) // no start with / or end with /
             .build();
@@ -188,7 +189,7 @@ public class PutHiveStreaming extends AbstractSessionFactoryProcessor {
             .displayName("Database Name")
             .description("The name of the database in which to put the data.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -197,7 +198,7 @@ public class PutHiveStreaming extends AbstractSessionFactoryProcessor {
             .displayName("Table Name")
             .description("The name of the database table in which to put the data.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -207,7 +208,7 @@ public class PutHiveStreaming extends AbstractSessionFactoryProcessor {
             .description("A comma-delimited list of column names on which the table has been partitioned. The order of values in this list must "
                     + "correspond exactly to the order of partition columns specified during the table creation.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("[^,]+(,[^,]+)*"))) // comma-separated list with non-empty entries
             .build();
 
@@ -241,7 +242,7 @@ public class PutHiveStreaming extends AbstractSessionFactoryProcessor {
             .defaultValue("60")
             .required(true)
             .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor TXNS_PER_BATCH = new PropertyDescriptor.Builder()
@@ -249,7 +250,7 @@ public class PutHiveStreaming extends AbstractSessionFactoryProcessor {
             .displayName("Transactions per Batch")
             .description("A hint to Hive Streaming indicating how many transactions the processor task will need. This value must be greater than 1.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(GREATER_THAN_ONE_VALIDATOR)
             .defaultValue("100")
             .build();
@@ -259,7 +260,7 @@ public class PutHiveStreaming extends AbstractSessionFactoryProcessor {
             .displayName("Records per Transaction")
             .description("Number of records to process before committing the transaction. This value must be greater than 1.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(GREATER_THAN_ONE_VALIDATOR)
             .defaultValue("10000")
             .build();
@@ -272,7 +273,7 @@ public class PutHiveStreaming extends AbstractSessionFactoryProcessor {
             .defaultValue("0")
             .required(true)
             .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor ROLLBACK_ON_FAILURE = RollbackOnFailure.createRollbackOnFailureProperty(

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/SelectHiveQL.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/SelectHiveQL.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/SelectHiveQL.java
index 832636b..0bbddf5 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/SelectHiveQL.java
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/SelectHiveQL.java
@@ -45,6 +45,7 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.dbcp.hive.HiveDBCPService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.logging.ComponentLog;
@@ -109,7 +110,7 @@ public class SelectHiveQL extends AbstractHiveQLProcessor {
             .description("HiveQL SELECT query to execute. If this is not set, the query is assumed to be in the content of an incoming FlowFile.")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor FETCH_SIZE = new PropertyDescriptor.Builder()
@@ -120,7 +121,7 @@ public class SelectHiveQL extends AbstractHiveQLProcessor {
             .defaultValue("0")
             .required(true)
             .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor MAX_ROWS_PER_FLOW_FILE = new PropertyDescriptor.Builder()
@@ -131,7 +132,7 @@ public class SelectHiveQL extends AbstractHiveQLProcessor {
             .defaultValue("0")
             .required(true)
             .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor MAX_FRAGMENTS = new PropertyDescriptor.Builder()
@@ -142,7 +143,7 @@ public class SelectHiveQL extends AbstractHiveQLProcessor {
             .defaultValue("0")
             .required(true)
             .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor HIVEQL_CSV_HEADER = new PropertyDescriptor.Builder()
@@ -161,7 +162,7 @@ public class SelectHiveQL extends AbstractHiveQLProcessor {
             .description("Comma separated list of header fields")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor HIVEQL_CSV_DELIMITER = new PropertyDescriptor.Builder()
@@ -171,7 +172,7 @@ public class SelectHiveQL extends AbstractHiveQLProcessor {
             .required(true)
             .defaultValue(",")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor HIVEQL_CSV_QUOTE = new PropertyDescriptor.Builder()
@@ -200,7 +201,7 @@ public class SelectHiveQL extends AbstractHiveQLProcessor {
             .required(true)
             .allowableValues(AVRO, CSV)
             .defaultValue(AVRO)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
 
     private final static List<PropertyDescriptor> propertyDescriptors;


[05/15] nifi git commit: NIFI-4149 - Indicate if EL is evaluated against FFs or not - take into account input requirement for documentation rendering - Renamed variable registry scope and added comments - Doc + change in mock framework to check scope + u

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertJSONToSQL.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertJSONToSQL.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertJSONToSQL.java
index 9705dc0..272a3ff 100755
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertJSONToSQL.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertJSONToSQL.java
@@ -51,6 +51,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.dbcp.DBCPService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -139,21 +140,21 @@ public class ConvertJSONToSQL extends AbstractProcessor {
             .name("Table Name")
             .description("The name of the table that the statement should update")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
     static final PropertyDescriptor CATALOG_NAME = new PropertyDescriptor.Builder()
             .name("Catalog Name")
             .description("The name of the catalog that the statement should update. This may not apply for the database that you are updating. In this case, leave the field empty")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
     static final PropertyDescriptor SCHEMA_NAME = new PropertyDescriptor.Builder()
             .name("Schema Name")
             .description("The name of the schema that the table belongs to. This may not apply for the database that you are updating. In this case, leave the field empty")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
     static final PropertyDescriptor TRANSLATE_FIELD_NAMES = new PropertyDescriptor.Builder()
@@ -183,7 +184,7 @@ public class ConvertJSONToSQL extends AbstractProcessor {
                     + "This property is ignored if the Statement Type is INSERT")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     static final PropertyDescriptor QUOTED_IDENTIFIERS = new PropertyDescriptor.Builder()
@@ -208,7 +209,7 @@ public class ConvertJSONToSQL extends AbstractProcessor {
             .name("jts-sql-param-attr-prefix")
             .displayName("SQL Parameter Attribute Prefix")
             .description("The string to be prepended to the outgoing flow file attributes, such as <sql>.args.1.value, where <sql> is replaced with the specified value")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
             .required(true)
             .defaultValue("sql")

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DetectDuplicate.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DetectDuplicate.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DetectDuplicate.java
index 2fd7b80..b8a6adf 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DetectDuplicate.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DetectDuplicate.java
@@ -41,6 +41,7 @@ import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
 import org.apache.nifi.distributed.cache.client.Serializer;
 import org.apache.nifi.distributed.cache.client.exception.DeserializationException;
 import org.apache.nifi.distributed.cache.client.exception.SerializationException;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
@@ -81,7 +82,7 @@ public class DetectDuplicate extends AbstractProcessor {
             .required(true)
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(ResultType.STRING, true))
             .defaultValue("${hash.value}")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     public static final PropertyDescriptor FLOWFILE_DESCRIPTION = new PropertyDescriptor.Builder()
             .name("FlowFile Description")
@@ -90,7 +91,7 @@ public class DetectDuplicate extends AbstractProcessor {
                     + ORIGINAL_DESCRIPTION_ATTRIBUTE_NAME + "\" attribute")
             .required(true)
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(ResultType.STRING, true))
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("")
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java
index fa3d1b6..b4f33eb 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java
@@ -31,6 +31,7 @@ import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -106,7 +107,7 @@ public class EnforceOrder extends AbstractProcessor {
                 " If evaluated result is empty, the FlowFile will be routed to failure.")
         .required(true)
         .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .defaultValue("${filename}")
         .build();
 
@@ -117,7 +118,7 @@ public class EnforceOrder extends AbstractProcessor {
                 " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.")
         .required(true)
         .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .build();
 
     public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder()
@@ -129,7 +130,7 @@ public class EnforceOrder extends AbstractProcessor {
                 " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.")
         .required(true)
         .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .defaultValue("0")
         .build();
 
@@ -143,7 +144,7 @@ public class EnforceOrder extends AbstractProcessor {
                 " and maximum order will be left unknown until consecutive FlowFiles provide a valid maximum order.")
         .required(false)
         .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .build();
 
     public static final PropertyDescriptor WAIT_TIMEOUT = new PropertyDescriptor.Builder()
@@ -153,7 +154,7 @@ public class EnforceOrder extends AbstractProcessor {
         .required(true)
         .defaultValue("10 min")
         .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .build();
 
     public static final PropertyDescriptor INACTIVE_TIMEOUT = new PropertyDescriptor.Builder()
@@ -169,7 +170,7 @@ public class EnforceOrder extends AbstractProcessor {
         .required(true)
         .defaultValue("30 min")
         .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .build();
 
     public static final PropertyDescriptor BATCH_COUNT = new PropertyDescriptor.Builder()
@@ -179,7 +180,7 @@ public class EnforceOrder extends AbstractProcessor {
         .required(true)
         .defaultValue("1000")
         .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .build();
 
     public static final Relationship REL_SUCCESS = new Relationship.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
index cddfceb..0f310d7 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
@@ -47,6 +47,7 @@ import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.ProcessContext;
@@ -201,17 +202,22 @@ public class EvaluateJsonPath extends AbstractJsonPathProcessor {
 
     @Override
     protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
-        return new PropertyDescriptor.Builder().name(propertyDescriptorName).expressionLanguageSupported(false).addValidator(new JsonPathValidator() {
-            @Override
-            public void cacheComputedValue(String subject, String input, JsonPath computedJsonPath) {
-                cachedJsonPathMap.put(input, computedJsonPath);
-            }
-
-            @Override
-            public boolean isStale(String subject, String input) {
-                return cachedJsonPathMap.get(input) == null;
-            }
-        }).required(false).dynamic(true).build();
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+                .addValidator(
+                        new JsonPathValidator() {
+                            @Override
+                            public void cacheComputedValue(String subject, String input, JsonPath computedJsonPath) {
+                                cachedJsonPathMap.put(input, computedJsonPath);
+                            }
+
+                            @Override
+                            public boolean isStale(String subject, String input) {
+                                return cachedJsonPathMap.get(input) == null;
+                            }
+                })
+                .required(false).dynamic(true).build();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java
index 08690ba..ae50206 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java
@@ -19,6 +19,8 @@ package org.apache.nifi.processors.standard;
 import static javax.xml.xpath.XPathConstants.NODESET;
 import static javax.xml.xpath.XPathConstants.STRING;
 
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -65,6 +67,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -75,8 +78,6 @@ import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.processor.io.OutputStreamCallback;
-import org.apache.nifi.stream.io.BufferedInputStream;
-import org.apache.nifi.stream.io.BufferedOutputStream;
 import org.xml.sax.EntityResolver;
 import org.xml.sax.InputSource;
 
@@ -221,8 +222,12 @@ public class EvaluateXPath extends AbstractProcessor {
     @Override
     protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
         return new PropertyDescriptor.Builder()
-                .name(propertyDescriptorName).expressionLanguageSupported(false)
-                .addValidator(new XPathValidator()).required(false).dynamic(true).build();
+                .name(propertyDescriptorName)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+                .addValidator(new XPathValidator())
+                .required(false)
+                .dynamic(true)
+                .build();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java
index cfbb48b..af81354 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java
@@ -55,6 +55,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -231,8 +232,13 @@ public class EvaluateXQuery extends AbstractProcessor {
 
     @Override
     protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
-        return new PropertyDescriptor.Builder().name(propertyDescriptorName).expressionLanguageSupported(false)
-                .addValidator(new XQueryValidator()).required(false).dynamic(true).build();
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+                .addValidator(new XQueryValidator())
+                .required(false)
+                .dynamic(true)
+                .build();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteProcess.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteProcess.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteProcess.java
index 9a07127..91cf1ba 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteProcess.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteProcess.java
@@ -31,6 +31,7 @@ import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.RequiredPermission;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -93,7 +94,7 @@ public class ExecuteProcess extends AbstractProcessor {
     .name("Command")
     .description("Specifies the command to be executed; if just the name of an executable is provided, it must be in the user's environment PATH.")
     .required(true)
-    .expressionLanguageSupported(true)
+    .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
     .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
     .build();
 
@@ -101,14 +102,14 @@ public class ExecuteProcess extends AbstractProcessor {
     .name("Command Arguments")
     .description("The arguments to supply to the executable delimited by white space. White space can be escaped by enclosing it in double-quotes.")
     .required(false)
-    .expressionLanguageSupported(true)
+    .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
     .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
     .build();
 
     public static final PropertyDescriptor WORKING_DIR = new PropertyDescriptor.Builder()
     .name("Working Directory")
     .description("The directory to use as the current working directory when executing the command")
-    .expressionLanguageSupported(true)
+    .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
     .addValidator(StandardValidators.createDirectoryExistsValidator(false, true))
     .required(false)
     .build();
@@ -119,7 +120,7 @@ public class ExecuteProcess extends AbstractProcessor {
             + "that the output will be captured for this amount of time and a FlowFile will then be sent out with the results "
             + "and a new FlowFile will be started, rather than waiting for the process to finish before sending out the results")
             .required(false)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
             .build();
 
@@ -130,7 +131,7 @@ public class ExecuteProcess extends AbstractProcessor {
             .required(false)
             .allowableValues("true", "false")
             .defaultValue("false")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java
index cb9388f..203d02a 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java
@@ -42,6 +42,7 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.dbcp.DBCPService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.logging.ComponentLog;
@@ -124,7 +125,7 @@ public class ExecuteSQL extends AbstractProcessor {
                     + "Language is not evaluated for flow file contents.")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor QUERY_TIMEOUT = new PropertyDescriptor.Builder()
@@ -196,8 +197,8 @@ public class ExecuteSQL extends AbstractProcessor {
         final Integer queryTimeout = context.getProperty(QUERY_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue();
         final boolean convertNamesForAvro = context.getProperty(NORMALIZE_NAMES_FOR_AVRO).asBoolean();
         final Boolean useAvroLogicalTypes = context.getProperty(USE_AVRO_LOGICAL_TYPES).asBoolean();
-        final Integer defaultPrecision = context.getProperty(DEFAULT_PRECISION).evaluateAttributeExpressions().asInteger();
-        final Integer defaultScale = context.getProperty(DEFAULT_SCALE).evaluateAttributeExpressions().asInteger();
+        final Integer defaultPrecision = context.getProperty(DEFAULT_PRECISION).evaluateAttributeExpressions(fileToProcess).asInteger();
+        final Integer defaultScale = context.getProperty(DEFAULT_SCALE).evaluateAttributeExpressions(fileToProcess).asInteger();
         final StopWatch stopWatch = new StopWatch(true);
         final String selectQuery;
         if (context.getProperty(SQL_SELECT_QUERY).isSet()) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteStreamCommand.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteStreamCommand.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteStreamCommand.java
index 84880f1..6f23ca8 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteStreamCommand.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteStreamCommand.java
@@ -16,6 +16,24 @@
  */
 package org.apache.nifi.processors.standard;
 
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.lang.ProcessBuilder.Redirect;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.annotation.behavior.DynamicProperty;
@@ -35,6 +53,7 @@ import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -50,24 +69,6 @@ import org.apache.nifi.processors.standard.util.ArgumentUtils;
 import org.apache.nifi.processors.standard.util.SoftLimitBoundedByteArrayOutputStream;
 import org.apache.nifi.stream.io.StreamUtils;
 
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.OutputStream;
-import java.lang.ProcessBuilder.Redirect;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicReference;
-
 /**
  * <p>
  * This processor executes an external command on the contents of a flow file, and creates a new flow file with the results of the command.
@@ -171,7 +172,7 @@ public class ExecuteStreamCommand extends AbstractProcessor {
     static final PropertyDescriptor EXECUTION_COMMAND = new PropertyDescriptor.Builder()
             .name("Command Path")
             .description("Specifies the command to be executed; if just the name of an executable is provided, it must be in the user's environment PATH.")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR)
             .required(true)
             .build();
@@ -179,7 +180,8 @@ public class ExecuteStreamCommand extends AbstractProcessor {
     static final PropertyDescriptor EXECUTION_ARGUMENTS = new PropertyDescriptor.Builder()
             .name("Command Arguments")
             .description("The arguments to supply to the executable delimited by the ';' character.")
-            .expressionLanguageSupported(true).addValidator(new Validator() {
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(new Validator() {
 
                 @Override
                 public ValidationResult validate(String subject, String input, ValidationContext context) {
@@ -200,7 +202,7 @@ public class ExecuteStreamCommand extends AbstractProcessor {
     static final PropertyDescriptor WORKING_DIR = new PropertyDescriptor.Builder()
             .name("Working Directory")
             .description("The directory to use as the current working directory when executing the command")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.createDirectoryExistsValidator(true, true))
             .required(false)
             .build();
@@ -466,7 +468,7 @@ public class ExecuteStreamCommand extends AbstractProcessor {
 
                     // Because the outputstream has a cap that the copy doesn't know about, adjust
                     // the actual size
-                    if (longSize > (long) attributeSize) { // Explicit cast for readability
+                    if (longSize > attributeSize) { // Explicit cast for readability
                         size = attributeSize;
                     } else{
                         size = (int) longSize; // Note: safe cast, longSize is limited by attributeSize

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java
index 692446b..87b05ff 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java
@@ -46,6 +46,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -261,7 +262,7 @@ public class ExtractText extends AbstractProcessor {
     protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
         return new PropertyDescriptor.Builder()
                 .name(propertyDescriptorName)
-                .expressionLanguageSupported(false)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE)
                 .addValidator(StandardValidators.createRegexValidator(0, 40, true))
                 .required(false)
                 .dynamic(true)

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchDistributedMapCache.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchDistributedMapCache.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchDistributedMapCache.java
index e50e843..7e9cedc 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchDistributedMapCache.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchDistributedMapCache.java
@@ -35,6 +35,7 @@ import org.apache.nifi.distributed.cache.client.Serializer;
 import org.apache.nifi.distributed.cache.client.exception.DeserializationException;
 import org.apache.nifi.distributed.cache.client.exception.SerializationException;
 import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
@@ -92,7 +93,7 @@ public class FetchDistributedMapCache extends AbstractProcessor {
             .required(true)
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(ResultType.STRING, true))
             .defaultValue("${hash.value}")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor PROP_PUT_CACHE_VALUE_IN_ATTRIBUTE = new PropertyDescriptor.Builder()
@@ -101,7 +102,7 @@ public class FetchDistributedMapCache extends AbstractProcessor {
                     + "FlowFile. The attribute key to put to is determined by evaluating value of this property. If multiple Cache Entry Identifiers are selected, "
                     + "multiple attributes will be written, using the evaluated value of this property, appended by a period (.) and the name of the cache entry identifier.")
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING))
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor PROP_PUT_ATTRIBUTE_MAX_LENGTH = new PropertyDescriptor.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFile.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFile.java
index 3d5899c..3bf3f52 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFile.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFile.java
@@ -30,6 +30,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.RequiredPermission;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.LogLevel;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -86,14 +87,14 @@ public class FetchFile extends AbstractProcessor {
         .name("File to Fetch")
         .description("The fully-qualified filename of the file to fetch from the file system")
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .defaultValue("${absolute.path}/${filename}")
         .required(true)
         .build();
     static final PropertyDescriptor COMPLETION_STRATEGY = new PropertyDescriptor.Builder()
         .name("Completion Strategy")
         .description("Specifies what to do with the original file on the file system once it has been pulled into NiFi")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues(COMPLETION_NONE, COMPLETION_MOVE, COMPLETION_DELETE)
         .defaultValue(COMPLETION_NONE.getValue())
         .required(true)
@@ -102,7 +103,7 @@ public class FetchFile extends AbstractProcessor {
         .name("Move Destination Directory")
         .description("The directory to the move the original file to once it has been fetched from the file system. This property is ignored unless the Completion Strategy is set to \"Move File\". "
             + "If the directory does not exist, it will be created.")
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .required(false)
         .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java
index b92ed98..3ba0066 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java
@@ -21,6 +21,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -67,21 +68,21 @@ public abstract class FetchFileTransfer extends AbstractProcessor {
         .name("Hostname")
         .description("The fully-qualified hostname or IP address of the host to fetch the data from")
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .required(true)
         .build();
     static final PropertyDescriptor UNDEFAULTED_PORT = new PropertyDescriptor.Builder()
         .name("Port")
         .description("The port to connect to on the remote host to fetch the data from")
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .required(true)
         .build();
     public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
         .name("Username")
         .description("Username")
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .required(true)
         .build();
     public static final PropertyDescriptor REMOTE_FILENAME = new PropertyDescriptor.Builder()
@@ -89,13 +90,13 @@ public abstract class FetchFileTransfer extends AbstractProcessor {
         .description("The fully qualified filename on the remote system")
         .required(true)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .build();
     static final PropertyDescriptor COMPLETION_STRATEGY = new PropertyDescriptor.Builder()
         .name("Completion Strategy")
         .description("Specifies what to do with the original file on the server once it has been pulled into NiFi. If the Completion Strategy fails, a warning will be "
             + "logged but the data will still be transferred.")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues(COMPLETION_NONE, COMPLETION_MOVE, COMPLETION_DELETE)
         .defaultValue(COMPLETION_NONE.getValue())
         .required(true)
@@ -105,7 +106,7 @@ public abstract class FetchFileTransfer extends AbstractProcessor {
         .description("The directory on the remote server to the move the original file to once it has been ingested into NiFi. "
             + "This property is ignored unless the Completion Strategy is set to \"Move File\". The specified directory must already exist on"
             + "the remote system, or the rename will fail.")
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .required(false)
         .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FlattenJson.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FlattenJson.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FlattenJson.java
index afb6747..ecc9a64 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FlattenJson.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FlattenJson.java
@@ -28,6 +28,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.expression.ExpressionLanguageCompiler;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -90,7 +91,7 @@ public class FlattenJson extends AbstractProcessor {
 
                 return new ValidationResult.Builder().subject(subject).input(input).valid(valid).explanation(message).build();
             })
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final AllowableValue FLATTEN_MODE_NORMAL = new AllowableValue("normal", "normal",
@@ -109,7 +110,7 @@ public class FlattenJson extends AbstractProcessor {
             .defaultValue(FLATTEN_MODE_KEEP_ARRAYS.getValue())
             .required(true)
             .allowableValues(FLATTEN_MODE_NORMAL, FLATTEN_MODE_KEEP_ARRAYS, FLATTEN_MODE_DOT_NOTATION)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
 
     private List<PropertyDescriptor> properties;

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateFlowFile.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateFlowFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateFlowFile.java
index 105ac4a..ba3291d 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateFlowFile.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateFlowFile.java
@@ -41,6 +41,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.DataUnit;
@@ -56,7 +57,8 @@ import org.apache.nifi.processor.util.StandardValidators;
 @InputRequirement(Requirement.INPUT_FORBIDDEN)
 @CapabilityDescription("This processor creates FlowFiles with random data or custom content. GenerateFlowFile is useful" +
         "for load testing, configuration, and simulation.")
-@DynamicProperty(name = "Generated FlowFile attribute name", value = "Generated FlowFile attribute value", supportsExpressionLanguage = true,
+@DynamicProperty(name = "Generated FlowFile attribute name", value = "Generated FlowFile attribute value",
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
         description = "Specifies an attribute on generated FlowFiles defined by the Dynamic Property's key and value." +
         " If Expression Language is used, evaluation will be performed only once per batch of generated FlowFiles.")
 public class GenerateFlowFile extends AbstractProcessor {
@@ -102,7 +104,7 @@ public class GenerateFlowFile extends AbstractProcessor {
                     + "FlowFiles and the File Size will be ignored. Finally, if Expression Language is used, evaluation will be performed only once "
                     + "per batch of generated FlowFiles")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
     public static final PropertyDescriptor CHARSET = new PropertyDescriptor.Builder()
@@ -151,7 +153,7 @@ public class GenerateFlowFile extends AbstractProcessor {
             .required(false)
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
             .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .dynamic(true)
             .build();
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateTableFetch.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateTableFetch.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateTableFetch.java
index 188e282..d126de1 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateTableFetch.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateTableFetch.java
@@ -35,6 +35,7 @@ import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.components.state.StateManager;
 import org.apache.nifi.components.state.StateMap;
 import org.apache.nifi.dbcp.DBCPService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.ProcessContext;
@@ -95,8 +96,9 @@ import java.util.stream.IntStream;
         @WritesAttribute(attribute = "generatetablefetch.limit", description = "The number of result rows to be fetched by the SQL statement."),
         @WritesAttribute(attribute = "generatetablefetch.offset", description = "Offset to be used to retrieve the corresponding partition.")
 })
-@DynamicProperty(name = "Initial Max Value", value = "Attribute Expression Language", supportsExpressionLanguage = false, description = "Specifies an initial "
-        + "max value for max value columns. Properties should be added in the format `initial.maxvalue.{max_value_column}`.")
+@DynamicProperty(name = "Initial Max Value", value = "Attribute Expression Language",
+                 expressionLanguageScope = ExpressionLanguageScope.NONE, description = "Specifies an initial "
+                         + "max value for max value columns. Properties should be added in the format `initial.maxvalue.{max_value_column}`.")
 public class GenerateTableFetch extends AbstractDatabaseFetchProcessor {
 
     public static final PropertyDescriptor PARTITION_SIZE = new PropertyDescriptor.Builder()
@@ -108,7 +110,7 @@ public class GenerateTableFetch extends AbstractDatabaseFetchProcessor {
                     + "in the table.")
             .defaultValue("10000")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFile.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFile.java
index a53ff8e..07f78e2 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFile.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFile.java
@@ -29,6 +29,7 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.logging.ComponentLog;
@@ -112,7 +113,7 @@ public class GetFile extends AbstractProcessor {
             .description("The input directory from which to pull files")
             .required(true)
             .addValidator(StandardValidators.createDirectoryExistsValidator(true, false))
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
     public static final PropertyDescriptor RECURSE = new PropertyDescriptor.Builder()
             .name("Recurse Subdirectories")

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java
index 9914ad7..ec5fc2c 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java
@@ -82,6 +82,7 @@ import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.components.state.StateManager;
 import org.apache.nifi.components.state.StateMap;
 import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.logging.ComponentLog;
@@ -107,8 +108,9 @@ import org.apache.nifi.util.Tuple;
     + "management, stored \"last modified\" and etag fields never expire. If the URL in GetHttp uses Expression Language that is unbounded, there "
     + "is the potential for Out of Memory Errors to occur.")
 @DynamicProperties({
-    @DynamicProperty(name = "Header Name", value = "The Expression Language to be used to populate the header value", description = "The additional headers to be sent by the processor " +
-            "whenever making a new HTTP request. \n " +
+    @DynamicProperty(name = "Header Name", value = "The Expression Language to be used to populate the header value",
+            expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
+            description = "The additional headers to be sent by the processor whenever making a new HTTP request. \n " +
             "Setting a dynamic property name to XYZ and value to ${attribute} will result in the header 'XYZ: attribute_value' being sent to the HTTP endpoint"),
 })
 @WritesAttributes({
@@ -131,7 +133,7 @@ public class GetHTTP extends AbstractSessionFactoryProcessor {
             .name("URL")
             .description("The URL to pull from")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.URL_VALIDATOR)
             .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("https?\\://.*")))
             .build();
@@ -164,7 +166,7 @@ public class GetHTTP extends AbstractSessionFactoryProcessor {
     public static final PropertyDescriptor FILENAME = new PropertyDescriptor.Builder()
             .name("Filename")
             .description("The filename to assign to the file when pulled")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING))
             .required(true)
             .build();
@@ -328,7 +330,7 @@ public class GetHTTP extends AbstractSessionFactoryProcessor {
     protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
         return new PropertyDescriptor.Builder()
                 .name(propertyDescriptorName)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
                 .addValidator(Validator.VALID)
                 .required(false)
                 .dynamic(true)

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java
index ad1b171..b2a7ae2 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java
@@ -28,6 +28,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.http.HttpContextMap;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -128,7 +129,7 @@ public class HandleHttpRequest extends AbstractProcessor {
             .description("The Port to listen on for incoming HTTP requests")
             .required(true)
             .addValidator(StandardValidators.createLongValidator(0L, 65535L, true))
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .defaultValue("80")
             .build();
     public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor.Builder()
@@ -136,7 +137,7 @@ public class HandleHttpRequest extends AbstractProcessor {
             .description("The Hostname to bind to. If not specified, will bind to all hosts")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
     public static final PropertyDescriptor HTTP_CONTEXT_MAP = new PropertyDescriptor.Builder()
             .name("HTTP Context Map")
@@ -165,7 +166,7 @@ public class HandleHttpRequest extends AbstractProcessor {
                     + "404: NotFound")
             .required(false)
             .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
     public static final PropertyDescriptor ALLOW_GET = new PropertyDescriptor.Builder()
             .name("Allow GET")
@@ -214,7 +215,7 @@ public class HandleHttpRequest extends AbstractProcessor {
             .description("A comma-separated list of non-standard HTTP Methods that should be allowed")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
     public static final PropertyDescriptor CLIENT_AUTH = new PropertyDescriptor.Builder()
             .name("Client Authentication")

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java
index 7fe0b71..2a5c1a6 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java
@@ -35,6 +35,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.http.HttpContextMap;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -50,7 +51,9 @@ import org.apache.nifi.util.StopWatch;
 @Tags({"http", "https", "response", "egress", "web service"})
 @CapabilityDescription("Sends an HTTP Response to the Requestor that generated a FlowFile. This Processor is designed to be used in conjunction with "
         + "the HandleHttpRequest in order to create a web service.")
-@DynamicProperty(name = "An HTTP header name", value = "An HTTP header value", description = "These HTTPHeaders are set in the HTTP Response")
+@DynamicProperty(name = "An HTTP header name", value = "An HTTP header value",
+                    description = "These HTTPHeaders are set in the HTTP Response",
+                    expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
 @ReadsAttributes({
     @ReadsAttribute(attribute = HTTPUtils.HTTP_CONTEXT_ID, description = "The value of this attribute is used to lookup the HTTP Response so that the "
         + "proper message can be sent back to the requestor. If this attribute is missing, the FlowFile will be routed to 'failure.'"),
@@ -69,7 +72,7 @@ public class HandleHttpResponse extends AbstractProcessor {
             .description("The HTTP Status Code to use when responding to the HTTP Request. See Section 10 of RFC 2616 for more information.")
             .required(true)
             .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     public static final PropertyDescriptor HTTP_CONTEXT_MAP = new PropertyDescriptor.Builder()
             .name("HTTP Context Map")
@@ -111,7 +114,7 @@ public class HandleHttpResponse extends AbstractProcessor {
                 .name(propertyDescriptorName)
                 .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
                 .dynamic(true)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
                 .build();
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
index 0f0f878..834dd9a 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
@@ -46,6 +46,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.logging.ComponentLog;
@@ -126,9 +127,9 @@ import static org.apache.commons.lang3.StringUtils.trimToEmpty;
     @WritesAttribute(attribute = "invokehttp.java.exception.message", description = "The Java exception message raised when the processor fails"),
     @WritesAttribute(attribute = "user-defined", description = "If the 'Put Response Body In Attribute' property is set then whatever it is set to "
         + "will become the attribute key and the value would be the body of the HTTP response.")})
-@DynamicProperty(name = "Header Name", value = "Attribute Expression Language", supportsExpressionLanguage = true, description = "Send request header "
-        + "with a key matching the Dynamic Property Key and a value created by evaluating the Attribute Expression Language set in the value "
-        + "of the Dynamic Property.")
+@DynamicProperty(name = "Header Name", value = "Attribute Expression Language", expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
+                    description = "Send request header with a key matching the Dynamic Property Key and a value created by evaluating "
+                            + "the Attribute Expression Language set in the value of the Dynamic Property.")
 public final class InvokeHTTP extends AbstractProcessor {
 
     // flowfile attribute keys returned after reading the response
@@ -163,7 +164,7 @@ public final class InvokeHTTP extends AbstractProcessor {
                 + "Methods other than POST, PUT and PATCH will be sent without a message body.")
             .required(true)
             .defaultValue("GET")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING))
             .build();
 
@@ -171,7 +172,7 @@ public final class InvokeHTTP extends AbstractProcessor {
             .name("Remote URL")
             .description("Remote URL which will be connected to, including scheme, host, port, path.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.URL_VALIDATOR)
             .build();
 
@@ -241,7 +242,7 @@ public final class InvokeHTTP extends AbstractProcessor {
             .description("The fully qualified hostname or IP address of the proxy server")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor PROP_PROXY_PORT = new PropertyDescriptor.Builder()
@@ -249,7 +250,7 @@ public final class InvokeHTTP extends AbstractProcessor {
             .description("The port of the proxy server")
             .required(false)
             .addValidator(StandardValidators.PORT_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor PROP_PROXY_USER = new PropertyDescriptor.Builder()
@@ -258,7 +259,7 @@ public final class InvokeHTTP extends AbstractProcessor {
             .description("Username to set when authenticating against proxy")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor PROP_PROXY_PASSWORD = new PropertyDescriptor.Builder()
@@ -268,7 +269,7 @@ public final class InvokeHTTP extends AbstractProcessor {
             .required(false)
             .sensitive(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor PROP_CONTENT_TYPE = new PropertyDescriptor.Builder()
@@ -276,7 +277,7 @@ public final class InvokeHTTP extends AbstractProcessor {
             .description("The Content-Type to specify for when content is being transmitted through a PUT, POST or PATCH. "
                     + "In the case of an empty value after evaluating an expression language expression, Content-Type defaults to " + DEFAULT_CONTENT_TYPE)
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("${" + CoreAttributes.MIME_TYPE.key() + "}")
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING))
             .build();
@@ -324,7 +325,7 @@ public final class InvokeHTTP extends AbstractProcessor {
             .description("If set, the response body received back will be put into an attribute of the original FlowFile instead of a separate "
                     + "FlowFile. The attribute key to put to is determined by evaluating value of this property. ")
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING))
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor PROP_PUT_ATTRIBUTE_MAX_LENGTH = new PropertyDescriptor.Builder()
@@ -497,7 +498,7 @@ public final class InvokeHTTP extends AbstractProcessor {
                 .name(propertyDescriptorName)
                 .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
                 .dynamic(true)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
                 .build();
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/JoltTransformJSON.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/JoltTransformJSON.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/JoltTransformJSON.java
index 5856f81..69ebad2 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/JoltTransformJSON.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/JoltTransformJSON.java
@@ -42,6 +42,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.logging.ComponentLog;
@@ -96,7 +97,7 @@ public class JoltTransformJSON extends AbstractProcessor {
             .name("jolt-spec")
             .displayName("Jolt Specification")
             .description("Jolt Specification for transform of JSON data. This value is ignored if the Jolt Sort Transformation is selected.")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .required(false)
             .build();
@@ -106,7 +107,7 @@ public class JoltTransformJSON extends AbstractProcessor {
             .displayName("Custom Transformation Class Name")
             .description("Fully Qualified Class Name for Custom Transformation")
             .required(false)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -115,7 +116,7 @@ public class JoltTransformJSON extends AbstractProcessor {
             .displayName("Custom Module Directory")
             .description("Comma-separated list of paths to files and/or directories which contain modules containing custom transformations (that are not included on NiFi's classpath).")
             .required(false)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -123,7 +124,7 @@ public class JoltTransformJSON extends AbstractProcessor {
             .name("Transform Cache Size")
             .description("Compiling a Jolt Transform can be fairly expensive. Ideally, this will be done only once. However, if the Expression Language is used in the transform, we may need "
                 + "a new Transform for each FlowFile. This value controls how many of those Transforms we cache in memory in order to avoid having to compile the Transform each time.")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
             .defaultValue("1")
             .required(true)
@@ -299,7 +300,7 @@ public class JoltTransformJSON extends AbstractProcessor {
 
     private JoltTransform getTransform(final ProcessContext context, final FlowFile flowFile) throws Exception {
         final String specString;
-        if (context.getProperty(JOLT_SPEC).isSet() && !StringUtils.isEmpty(context.getProperty(JOLT_SPEC).getValue())) {
+        if (context.getProperty(JOLT_SPEC).isSet()) {
             specString = context.getProperty(JOLT_SPEC).evaluateAttributeExpressions(flowFile).getValue();
         } else {
             specString = null;

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFile.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFile.java
index 4cf01b0..8daa5fa 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFile.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFile.java
@@ -30,6 +30,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.ProcessContext;
@@ -116,7 +117,7 @@ public class ListFile extends AbstractListProcessor<FileInfo> {
             .description("The input directory from which files to pull files")
             .required(true)
             .addValidator(StandardValidators.createDirectoryExistsValidator(true, false))
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor RECURSE = new PropertyDescriptor.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFileTransfer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFileTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFileTransfer.java
index 3f35c4e..554fd4d 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFileTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFileTransfer.java
@@ -23,6 +23,7 @@ import java.text.SimpleDateFormat;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.util.StandardValidators;
@@ -42,20 +43,20 @@ public abstract class ListFileTransfer extends AbstractListProcessor<FileInfo> {
         .description("The fully qualified hostname or IP address of the remote system")
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .build();
     static final PropertyDescriptor UNDEFAULTED_PORT = new PropertyDescriptor.Builder()
         .name("Port")
         .description("The port to connect to on the remote host to fetch the data from")
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .required(true)
         .build();
     public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
         .name("Username")
         .description("Username")
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .required(true)
         .build();
     public static final PropertyDescriptor REMOTE_PATH = new PropertyDescriptor.Builder()
@@ -63,7 +64,7 @@ public abstract class ListFileTransfer extends AbstractListProcessor<FileInfo> {
         .description("The path on the remote system from which to pull or push files")
         .required(false)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .defaultValue(".")
         .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
index c441104..209e6d1 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
@@ -38,6 +38,7 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
 import org.apache.nifi.processor.DataUnit;
@@ -83,7 +84,7 @@ public class ListenHTTP extends AbstractSessionFactoryProcessor {
         .name("Base Path")
         .description("Base path for incoming connections")
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .defaultValue("contentListener")
         .addValidator(StandardValidators.URI_VALIDATOR)
         .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("(^[^/]+.*[^/]+$|^[^/]+$|^$)"))) // no start with / or end with /
@@ -92,7 +93,7 @@ public class ListenHTTP extends AbstractSessionFactoryProcessor {
         .name("Listening Port")
         .description("The Port to listen on for incoming connections")
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
         .build();
     public static final PropertyDescriptor AUTHORIZED_DN_PATTERN = new PropertyDescriptor.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDPRecord.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDPRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDPRecord.java
index 3c8e71b..2e3a04a 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDPRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDPRecord.java
@@ -29,6 +29,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.DataUnit;
@@ -89,7 +90,7 @@ public class ListenUDPRecord extends AbstractListenEventProcessor<StandardEvent>
             .description("IP, or name, of a remote host. Only Datagrams from the specified Sending Host Port and this host will "
                 + "be accepted. Improves Performance. May be a system property or an environment variable.")
             .addValidator(new HostValidator())
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor SENDING_HOST_PORT = new PropertyDescriptor.Builder()
@@ -98,7 +99,7 @@ public class ListenUDPRecord extends AbstractListenEventProcessor<StandardEvent>
             .description("Port being used by remote host to send Datagrams. Only Datagrams from the specified Sending Host and "
                 + "this port will be accepted. Improves Performance. May be a system property or an environment variable.")
             .addValidator(StandardValidators.PORT_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
@@ -106,7 +107,7 @@ public class ListenUDPRecord extends AbstractListenEventProcessor<StandardEvent>
             .displayName("Record Reader")
             .description("The Record Reader to use for reading the content of incoming datagrams.")
             .identifiesControllerService(RecordReaderFactory.class)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(true)
             .build();
 
@@ -115,7 +116,7 @@ public class ListenUDPRecord extends AbstractListenEventProcessor<StandardEvent>
             .displayName("Record Writer")
             .description("The Record Writer to use in order to serialize the data before writing to a flow file.")
             .identifiesControllerService(RecordSetWriterFactory.class)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(true)
             .build();
 
@@ -135,7 +136,7 @@ public class ListenUDPRecord extends AbstractListenEventProcessor<StandardEvent>
             .description("The maximum number of datagrams to write as records to a single FlowFile. The Batch Size will only be reached when " +
                     "data is coming in more frequently than the Poll Timeout.")
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .defaultValue("1000")
             .required(true)
             .build();


[12/15] nifi git commit: NIFI-4149 - Indicate if EL is evaluated against FFs or not - take into account input requirement for documentation rendering - Renamed variable registry scope and added comments - Doc + change in mock framework to check scope + u

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/ITDeleteAzureBlobStorage.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/ITDeleteAzureBlobStorage.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/ITDeleteAzureBlobStorage.java
index 4af6030..b7481d4 100644
--- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/ITDeleteAzureBlobStorage.java
+++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/ITDeleteAzureBlobStorage.java
@@ -16,18 +16,19 @@
  */
 package org.apache.nifi.processors.azure.storage;
 
-import com.microsoft.azure.storage.StorageException;
-import com.microsoft.azure.storage.blob.CloudBlobContainer;
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.security.InvalidKeyException;
+import java.util.UUID;
+
 import org.apache.nifi.processors.azure.AbstractAzureBlobStorageIT;
 import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.Test;
 
-import java.io.IOException;
-import java.net.URISyntaxException;
-import java.security.InvalidKeyException;
-import java.util.UUID;
+import com.microsoft.azure.storage.StorageException;
+import com.microsoft.azure.storage.blob.CloudBlobContainer;
 
 public class ITDeleteAzureBlobStorage extends AbstractAzureBlobStorageIT{
 
@@ -42,8 +43,6 @@ public class ITDeleteAzureBlobStorage extends AbstractAzureBlobStorageIT{
         final TestRunner runner = TestRunners.newTestRunner(DeleteAzureBlobStorage.class);
 
         try {
-            runner.setValidateExpressionUsage(true);
-
             runner.setProperty(AzureStorageUtils.ACCOUNT_NAME, AzureTestUtil.getAccountName());
             runner.setProperty(AzureStorageUtils.ACCOUNT_KEY, AzureTestUtil.getAccountKey());
             runner.setProperty(AzureStorageUtils.CONTAINER, containerName);

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/ITFetchAzureBlobStorage.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/ITFetchAzureBlobStorage.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/ITFetchAzureBlobStorage.java
index a9a2487..1810a08 100644
--- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/ITFetchAzureBlobStorage.java
+++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/ITFetchAzureBlobStorage.java
@@ -53,8 +53,6 @@ public class ITFetchAzureBlobStorage {
         final TestRunner runner = TestRunners.newTestRunner(new FetchAzureBlobStorage());
 
         try {
-            runner.setValidateExpressionUsage(true);
-
             runner.setProperty(AzureStorageUtils.ACCOUNT_NAME, AzureTestUtil.getAccountName());
             runner.setProperty(AzureStorageUtils.ACCOUNT_KEY, AzureTestUtil.getAccountKey());
             runner.setProperty(AzureStorageUtils.CONTAINER, containerName);

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/ITPutAzureStorageBlob.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/ITPutAzureStorageBlob.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/ITPutAzureStorageBlob.java
index 8b15d85..bfa1d4a 100644
--- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/ITPutAzureStorageBlob.java
+++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/ITPutAzureStorageBlob.java
@@ -42,8 +42,6 @@ public class ITPutAzureStorageBlob {
         final TestRunner runner = TestRunners.newTestRunner(new PutAzureBlobStorage());
 
         try {
-            runner.setValidateExpressionUsage(true);
-
             runner.setProperty(AzureStorageUtils.ACCOUNT_NAME, AzureTestUtil.getAccountName());
             runner.setProperty(AzureStorageUtils.ACCOUNT_KEY, AzureTestUtil.getAccountKey());
             runner.setProperty(AzureStorageUtils.CONTAINER, containerName);

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/AbstractCassandraProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/AbstractCassandraProcessor.java b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/AbstractCassandraProcessor.java
index 4b69e65..b0f82f9 100644
--- a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/AbstractCassandraProcessor.java
+++ b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/AbstractCassandraProcessor.java
@@ -33,6 +33,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -65,7 +66,7 @@ public abstract class AbstractCassandraProcessor extends AbstractProcessor {
                     + "comma-separated and in hostname:port format. Example node1:port,node2:port,...."
                     + " The default client port for Cassandra is 9042, but the port(s) must be explicitly specified.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.HOSTNAME_PORT_LIST_VALIDATOR)
             .build();
 
@@ -74,7 +75,7 @@ public abstract class AbstractCassandraProcessor extends AbstractProcessor {
             .description("The Cassandra Keyspace to connect to. If no keyspace is specified, the query will need to "
                     + "include the keyspace name before any table reference.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -100,7 +101,7 @@ public abstract class AbstractCassandraProcessor extends AbstractProcessor {
             .name("Username")
             .description("Username to access the Cassandra cluster")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -109,7 +110,7 @@ public abstract class AbstractCassandraProcessor extends AbstractProcessor {
             .description("Password to access the Cassandra cluster")
             .required(false)
             .sensitive(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -125,7 +126,7 @@ public abstract class AbstractCassandraProcessor extends AbstractProcessor {
             .name("Character Set")
             .description("Specifies the character set of the record data.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("UTF-8")
             .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/PutCassandraQL.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/PutCassandraQL.java b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/PutCassandraQL.java
index 8155021..ccedb1b 100644
--- a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/PutCassandraQL.java
+++ b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/PutCassandraQL.java
@@ -40,6 +40,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnShutdown;
 import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.ProcessContext;
@@ -98,7 +99,7 @@ public class PutCassandraQL extends AbstractCassandraProcessor {
                     + "Time Unit, such as: nanos, millis, secs, mins, hrs, days. A value of zero means there is no limit. ")
             .defaultValue("0 seconds")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/QueryCassandra.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/QueryCassandra.java b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/QueryCassandra.java
index b007c70..384c395 100644
--- a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/QueryCassandra.java
+++ b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/QueryCassandra.java
@@ -40,6 +40,7 @@ import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.lifecycle.OnShutdown;
 import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.annotation.behavior.WritesAttribute;
@@ -95,7 +96,7 @@ public class QueryCassandra extends AbstractCassandraProcessor {
             .description("CQL select query")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor QUERY_TIMEOUT = new PropertyDescriptor.Builder()
@@ -105,7 +106,7 @@ public class QueryCassandra extends AbstractCassandraProcessor {
                     + "Time Unit, such as: nanos, millis, secs, mins, hrs, days. A value of zero means there is no limit. ")
             .defaultValue("0 seconds")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
             .build();
 
@@ -115,7 +116,7 @@ public class QueryCassandra extends AbstractCassandraProcessor {
                     + "and means there is no limit.")
             .defaultValue("0")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.INTEGER_VALIDATOR)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/main/java/org/apache/nifi/cdc/mysql/processors/CaptureChangeMySQL.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/main/java/org/apache/nifi/cdc/mysql/processors/CaptureChangeMySQL.java b/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/main/java/org/apache/nifi/cdc/mysql/processors/CaptureChangeMySQL.java
index d15ad6b..8472d68 100644
--- a/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/main/java/org/apache/nifi/cdc/mysql/processors/CaptureChangeMySQL.java
+++ b/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/main/java/org/apache/nifi/cdc/mysql/processors/CaptureChangeMySQL.java
@@ -63,6 +63,7 @@ import org.apache.nifi.components.state.StateMap;
 import org.apache.nifi.distributed.cache.client.Deserializer;
 import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
 import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -174,7 +175,7 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
             .defaultValue("30 seconds")
             .required(true)
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor HOSTS = new PropertyDescriptor.Builder()
@@ -185,9 +186,8 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
                     + "the hosts in the list in order. If one node goes down and failover is enabled for the cluster, then the processor will connect "
                     + "to the active node (assuming its host entry is specified in this property.  The default port for MySQL connections is 3306.")
             .required(true)
-            .expressionLanguageSupported(false)
             .addValidator(StandardValidators.HOSTNAME_PORT_LIST_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor DRIVER_NAME = new PropertyDescriptor.Builder()
@@ -197,7 +197,7 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
             .defaultValue("com.mysql.jdbc.Driver")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor DRIVER_LOCATION = new PropertyDescriptor.Builder()
@@ -208,7 +208,7 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
             .defaultValue(null)
             .required(false)
             .addValidator(StandardValidators.createListValidator(true, true, StandardValidators.createURLorFileValidator()))
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
@@ -217,7 +217,7 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
             .description("Username to access the MySQL cluster")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
@@ -227,7 +227,7 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
             .required(false)
             .sensitive(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor SERVER_ID = new PropertyDescriptor.Builder()
@@ -238,7 +238,7 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
                     + "the replication group. If the Server ID is not specified, it defaults to 65535.")
             .required(false)
             .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor DIST_CACHE_CLIENT = new PropertyDescriptor.Builder()
@@ -297,7 +297,7 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
             .defaultValue("0 seconds")
             .required(true)
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor INIT_SEQUENCE_ID = new PropertyDescriptor.Builder()
@@ -309,7 +309,7 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
                     + "processor to guarantee ordered delivery of CDC events.")
             .required(false)
             .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor INIT_BINLOG_FILENAME = new PropertyDescriptor.Builder()
@@ -321,7 +321,7 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
                     + "Language is supported to enable the use of the Variable Registry and/or environment properties.")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor INIT_BINLOG_POSITION = new PropertyDescriptor.Builder()
@@ -334,7 +334,7 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
                     + "and/or environment properties.")
             .required(false)
             .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     private static List<PropertyDescriptor> propDescriptors;

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/src/main/java/org/apache/nifi/confluent/schemaregistry/ConfluentSchemaRegistry.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/src/main/java/org/apache/nifi/confluent/schemaregistry/ConfluentSchemaRegistry.java b/nifi-nar-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/src/main/java/org/apache/nifi/confluent/schemaregistry/ConfluentSchemaRegistry.java
index 3558286..2bf3a61 100644
--- a/nifi-nar-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/src/main/java/org/apache/nifi/confluent/schemaregistry/ConfluentSchemaRegistry.java
+++ b/nifi-nar-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/src/main/java/org/apache/nifi/confluent/schemaregistry/ConfluentSchemaRegistry.java
@@ -29,6 +29,7 @@ import org.apache.nifi.confluent.schemaregistry.client.SchemaRegistryClient;
 import org.apache.nifi.context.PropertyContext;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.schema.access.SchemaField;
 import org.apache.nifi.schema.access.SchemaNotFoundException;
@@ -68,7 +69,7 @@ public class ConfluentSchemaRegistry extends AbstractControllerService implement
         .name("url")
         .displayName("Schema Registry URLs")
         .description("A comma-separated list of URLs of the Schema Registry to interact with")
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .defaultValue("http://localhost:8081")
         .required(true)
         .addValidator(new MultipleURLValidator())
@@ -107,7 +108,7 @@ public class ConfluentSchemaRegistry extends AbstractControllerService implement
         .displayName("Communications Timeout")
         .description("Specifies how long to wait to receive data from the Schema Registry before considering the communications a failure")
         .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .defaultValue("30 secs")
         .required(true)
         .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/main/java/org/apache/nifi/processors/couchbase/AbstractCouchbaseProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/main/java/org/apache/nifi/processors/couchbase/AbstractCouchbaseProcessor.java b/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/main/java/org/apache/nifi/processors/couchbase/AbstractCouchbaseProcessor.java
index 49f83b8..4a04c16 100644
--- a/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/main/java/org/apache/nifi/processors/couchbase/AbstractCouchbaseProcessor.java
+++ b/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/main/java/org/apache/nifi/processors/couchbase/AbstractCouchbaseProcessor.java
@@ -25,6 +25,7 @@ import java.util.Set;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.couchbase.CouchbaseAttributes;
 import org.apache.nifi.couchbase.CouchbaseClusterControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -52,7 +53,7 @@ public abstract class AbstractCouchbaseProcessor extends AbstractProcessor {
 
     public static final PropertyDescriptor DOC_ID = new PropertyDescriptor.Builder().name("Document Id")
         .description("A static, fixed Couchbase document id, or an expression to construct the Couchbase document id.")
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/main/java/org/apache/nifi/processors/couchbase/GetCouchbaseKey.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/main/java/org/apache/nifi/processors/couchbase/GetCouchbaseKey.java b/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/main/java/org/apache/nifi/processors/couchbase/GetCouchbaseKey.java
index 96946b2..2692abc 100644
--- a/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/main/java/org/apache/nifi/processors/couchbase/GetCouchbaseKey.java
+++ b/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/main/java/org/apache/nifi/processors/couchbase/GetCouchbaseKey.java
@@ -94,7 +94,7 @@ public class GetCouchbaseKey extends AbstractCouchbaseProcessor {
         final long startNanos = System.nanoTime();
         final ComponentLog logger = getLogger();
         String docId = null;
-        if (!StringUtils.isEmpty(context.getProperty(DOC_ID).getValue())) {
+        if (context.getProperty(DOC_ID).isSet()) {
             docId = context.getProperty(DOC_ID).evaluateAttributeExpressions(inFile).getValue();
         } else {
             final byte[] content = new byte[(int) inFile.getSize()];

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/main/java/org/apache/nifi/processors/couchbase/PutCouchbaseKey.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/main/java/org/apache/nifi/processors/couchbase/PutCouchbaseKey.java b/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/main/java/org/apache/nifi/processors/couchbase/PutCouchbaseKey.java
index 7027fff..6cfe8f5 100644
--- a/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/main/java/org/apache/nifi/processors/couchbase/PutCouchbaseKey.java
+++ b/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/main/java/org/apache/nifi/processors/couchbase/PutCouchbaseKey.java
@@ -24,7 +24,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
@@ -120,7 +119,7 @@ public class PutCouchbaseKey extends AbstractCouchbaseProcessor {
         });
 
         String docId = flowFile.getAttribute(CoreAttributes.UUID.key());
-        if (!StringUtils.isEmpty(context.getProperty(DOC_ID).getValue())) {
+        if (context.getProperty(DOC_ID).isSet()) {
             docId = context.getProperty(DOC_ID).evaluateAttributeExpressions(flowFile).getValue();
         }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/test/java/org/apache/nifi/processors/couchbase/TestCouchbaseClusterService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/test/java/org/apache/nifi/processors/couchbase/TestCouchbaseClusterService.java b/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/test/java/org/apache/nifi/processors/couchbase/TestCouchbaseClusterService.java
index eb2220d..703bc5f 100644
--- a/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/test/java/org/apache/nifi/processors/couchbase/TestCouchbaseClusterService.java
+++ b/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/test/java/org/apache/nifi/processors/couchbase/TestCouchbaseClusterService.java
@@ -40,7 +40,6 @@ public class TestCouchbaseClusterService {
         System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.couchbase.TestCouchbaseClusterService", "debug");
 
         testRunner = TestRunners.newTestRunner(PutCouchbaseKey.class);
-        testRunner.setValidateExpressionUsage(false);
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/test/java/org/apache/nifi/processors/couchbase/TestGetCouchbaseKey.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/test/java/org/apache/nifi/processors/couchbase/TestGetCouchbaseKey.java b/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/test/java/org/apache/nifi/processors/couchbase/TestGetCouchbaseKey.java
index 92f999e..154be53 100644
--- a/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/test/java/org/apache/nifi/processors/couchbase/TestGetCouchbaseKey.java
+++ b/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/test/java/org/apache/nifi/processors/couchbase/TestGetCouchbaseKey.java
@@ -74,7 +74,6 @@ public class TestGetCouchbaseKey {
         System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.processors.couchbase.TestGetCouchbaseKey", "debug");
 
         testRunner = TestRunners.newTestRunner(GetCouchbaseKey.class);
-        testRunner.setValidateExpressionUsage(false);
     }
 
     private void setupMockBucket(Bucket bucket) throws InitializationException {

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/test/java/org/apache/nifi/processors/couchbase/TestPutCouchbaseKey.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/test/java/org/apache/nifi/processors/couchbase/TestPutCouchbaseKey.java b/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/test/java/org/apache/nifi/processors/couchbase/TestPutCouchbaseKey.java
index f870593..1599bf4 100644
--- a/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/test/java/org/apache/nifi/processors/couchbase/TestPutCouchbaseKey.java
+++ b/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/test/java/org/apache/nifi/processors/couchbase/TestPutCouchbaseKey.java
@@ -73,7 +73,6 @@ public class TestPutCouchbaseKey {
         System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.processors.couchbase.TestPutCouchbaseKey", "debug");
 
         testRunner = TestRunners.newTestRunner(PutCouchbaseKey.class);
-        testRunner.setValidateExpressionUsage(false);
     }
 
     private void setupMockBucket(Bucket bucket) throws InitializationException {

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/src/main/java/org/apache/nifi/reporting/datadog/DataDogReportingTask.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/src/main/java/org/apache/nifi/reporting/datadog/DataDogReportingTask.java b/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/src/main/java/org/apache/nifi/reporting/datadog/DataDogReportingTask.java
index fe8d650..bba9658 100644
--- a/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/src/main/java/org/apache/nifi/reporting/datadog/DataDogReportingTask.java
+++ b/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/src/main/java/org/apache/nifi/reporting/datadog/DataDogReportingTask.java
@@ -16,7 +16,6 @@
  */
 package org.apache.nifi.reporting.datadog;
 
-
 import com.codahale.metrics.Gauge;
 import com.codahale.metrics.MetricRegistry;
 import com.google.common.base.Optional;
@@ -34,6 +33,7 @@ import org.apache.nifi.controller.status.ConnectionStatus;
 import org.apache.nifi.controller.status.PortStatus;
 import org.apache.nifi.controller.status.ProcessGroupStatus;
 import org.apache.nifi.controller.status.ProcessorStatus;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.reporting.AbstractReportingTask;
 import org.apache.nifi.reporting.ReportingContext;
@@ -48,8 +48,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
 
-
-
 @Tags({"reporting", "datadog", "metrics"})
 @CapabilityDescription("Publishes metrics from NiFi to datadog. For accurate and informative reporting, components should have unique names.")
 public class DataDogReportingTask extends AbstractReportingTask {
@@ -73,7 +71,6 @@ public class DataDogReportingTask extends AbstractReportingTask {
     static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
             .name("API key")
             .description("Datadog API key. If specified value is 'agent', local Datadog agent will be used.")
-            .expressionLanguageSupported(false)
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
@@ -82,7 +79,7 @@ public class DataDogReportingTask extends AbstractReportingTask {
             .name("Metrics prefix")
             .description("Prefix to be added before every metric")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue("nifi")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
@@ -92,7 +89,7 @@ public class DataDogReportingTask extends AbstractReportingTask {
             .description("Environment, dataflow is running in. " +
                     "This property will be included as metrics tag.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue("dev")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/druid/DruidTranquilityController.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/druid/DruidTranquilityController.java b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/druid/DruidTranquilityController.java
index e5af8ea..158579a 100644
--- a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/druid/DruidTranquilityController.java
+++ b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/druid/DruidTranquilityController.java
@@ -44,6 +44,7 @@ import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.api.druid.DruidTranquilityService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.codehaus.jackson.map.ObjectMapper;
@@ -96,7 +97,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
             .description("A data source is the Druid equivalent of a database table.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor ZOOKEEPER_CONNECTION_STRING = new PropertyDescriptor.Builder()
@@ -105,7 +106,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
             .description("A comma-separated list of host:port pairs, each corresponding to a ZooKeeper server. Ex: localhost:2181")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor ZOOKEEPER_RETRY_BASE_SLEEP_TIME = new PropertyDescriptor.Builder()
@@ -114,7 +115,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
             .description("When a connection to Zookeeper needs to be retried, this property specifies the amount of time (in milliseconds) to wait at first before retrying.")
             .required(true)
             .defaultValue("1000")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
             .build();
 
@@ -124,7 +125,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
             .description("When a connection to Zookeeper needs to be retried, this property specifies how many times to attempt reconnection.")
             .required(true)
             .defaultValue("20")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
             .build();
 
@@ -134,7 +135,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
             .description("When a connection to Zookeeper needs to be retried, this property specifies the amount of time to sleep (in milliseconds) between retries.")
             .required(true)
             .defaultValue("30000")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
             .build();
 
@@ -145,7 +146,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
             .required(true)
             .defaultValue("druid/overlord")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor DRUID_DISCOVERY_PATH = new PropertyDescriptor.Builder()
@@ -155,7 +156,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
             .required(true)
             .defaultValue("/druid/discovery")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor CLUSTER_PARTITIONS = new PropertyDescriptor.Builder()
@@ -164,7 +165,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
             .description("The number of partitions in the Druid cluster.")
             .required(true)
             .defaultValue("1")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
             .build();
 
@@ -174,7 +175,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
             .description("The replication factor for the Druid cluster.")
             .required(true)
             .defaultValue("1")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
             .build();
 
@@ -185,7 +186,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
             .required(true)
             .defaultValue("timestamp")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor AGGREGATOR_JSON = new PropertyDescriptor.Builder()
@@ -226,7 +227,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
                     return new ValidationResult.Builder().subject(subject).input(value).valid(false).explanation(subject + " is not valid Aggregator JSON").build();
                 }
             })
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor DIMENSIONS_LIST = new PropertyDescriptor.Builder()
@@ -235,7 +236,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
             .description("A comma separated list of field names that will be stored as dimensions on ingest.")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor SEGMENT_GRANULARITY = new PropertyDescriptor.Builder()
@@ -285,7 +286,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
             .required(true)
             .defaultValue("2000")
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor MAX_PENDING_BATCHES = new PropertyDescriptor.Builder()
@@ -295,7 +296,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
             .required(true)
             .defaultValue("5")
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor LINGER_MILLIS = new PropertyDescriptor.Builder()
@@ -307,7 +308,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
             .required(true)
             .defaultValue("1000")
             .addValidator(StandardValidators.INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     private static final List<PropertyDescriptor> properties;
@@ -381,7 +382,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
         final String windowPeriod = context.getProperty(WINDOW_PERIOD).getValue();
         final String indexRetryPeriod = context.getProperty(INDEX_RETRY_PERIOD).getValue();
         final String aggregatorJSON = context.getProperty(AGGREGATOR_JSON).evaluateAttributeExpressions().getValue();
-        final String dimensionsStringList = context.getProperty(DIMENSIONS_LIST).getValue();
+        final String dimensionsStringList = context.getProperty(DIMENSIONS_LIST).evaluateAttributeExpressions().getValue();
         final int maxBatchSize = context.getProperty(MAX_BATCH_SIZE).evaluateAttributeExpressions().asInteger();
         final int maxPendingBatches = context.getProperty(MAX_PENDING_BATCHES).evaluateAttributeExpressions().asInteger();
         final int lingerMillis = context.getProperty(LINGER_MILLIS).evaluateAttributeExpressions().asInteger();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-processors/src/main/java/org/apache/nifi/processors/druid/PutDruidRecord.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-processors/src/main/java/org/apache/nifi/processors/druid/PutDruidRecord.java b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-processors/src/main/java/org/apache/nifi/processors/druid/PutDruidRecord.java
index 19e47a9..fa6cfac 100644
--- a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-processors/src/main/java/org/apache/nifi/processors/druid/PutDruidRecord.java
+++ b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-processors/src/main/java/org/apache/nifi/processors/druid/PutDruidRecord.java
@@ -51,6 +51,8 @@ import org.apache.nifi.serialization.RecordReader;
 import org.apache.nifi.serialization.RecordReaderFactory;
 
 import org.apache.nifi.controller.api.druid.DruidTranquilityService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+
 import com.metamx.tranquility.tranquilizer.Tranquilizer;
 import org.apache.nifi.serialization.RecordSetWriter;
 import org.apache.nifi.serialization.RecordSetWriterFactory;
@@ -86,7 +88,7 @@ public class PutDruidRecord extends AbstractSessionFactoryProcessor {
             .displayName("Record Writer")
             .description("The Record Writer to use in order to serialize the data to outgoing relationships.")
             .identifiesControllerService(RecordSetWriterFactory.class)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(true)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearch5Processor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearch5Processor.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearch5Processor.java
index 2e060a5..5017d2c 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearch5Processor.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearch5Processor.java
@@ -19,6 +19,7 @@ package org.apache.nifi.processors.elasticsearch;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.exception.ProcessException;
@@ -51,7 +52,7 @@ abstract class AbstractElasticsearch5Processor extends AbstractProcessor {
             .required(true)
             .defaultValue("UTF-8")
             .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
@@ -60,7 +61,7 @@ abstract class AbstractElasticsearch5Processor extends AbstractProcessor {
             .description("Username to access the Elasticsearch cluster")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearch5TransportClientProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearch5TransportClientProcessor.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearch5TransportClientProcessor.java
index 6449093..4a356b3 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearch5TransportClientProcessor.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearch5TransportClientProcessor.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.processors.elasticsearch;
 
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.exception.ProcessException;
@@ -51,7 +52,7 @@ abstract class AbstractElasticsearch5TransportClientProcessor extends AbstractEl
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .defaultValue("elasticsearch")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     protected static final PropertyDescriptor HOSTS = new PropertyDescriptor.Builder()
@@ -61,7 +62,7 @@ abstract class AbstractElasticsearch5TransportClientProcessor extends AbstractEl
                     + "host1:port,host2:port,....  For example testcluster:9300. This processor uses the Transport Client to "
                     + "connect to hosts. The default transport client port is 9300.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.HOSTNAME_PORT_LIST_VALIDATOR)
             .build();
 
@@ -75,7 +76,7 @@ abstract class AbstractElasticsearch5TransportClientProcessor extends AbstractEl
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .dynamicallyModifiesClasspath(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     protected static final PropertyDescriptor PING_TIMEOUT = new PropertyDescriptor.Builder()
@@ -86,7 +87,7 @@ abstract class AbstractElasticsearch5TransportClientProcessor extends AbstractEl
             .required(true)
             .defaultValue("5s")
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     protected static final PropertyDescriptor SAMPLER_INTERVAL = new PropertyDescriptor.Builder()
@@ -97,7 +98,7 @@ abstract class AbstractElasticsearch5TransportClientProcessor extends AbstractEl
             .required(true)
             .defaultValue("5s")
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     protected final AtomicReference<Client> esClient = new AtomicReference<>();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/DeleteElasticsearch5.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/DeleteElasticsearch5.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/DeleteElasticsearch5.java
index 9e77775..f4a6dba 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/DeleteElasticsearch5.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/DeleteElasticsearch5.java
@@ -26,6 +26,7 @@ import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.ProcessContext;
@@ -83,7 +84,7 @@ public class DeleteElasticsearch5 extends AbstractElasticsearch5TransportClientP
             .displayName("Document Identifier")
             .description("The identifier for the document to be deleted")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -92,7 +93,7 @@ public class DeleteElasticsearch5 extends AbstractElasticsearch5TransportClientP
             .displayName("Index")
             .description("The name of the index to delete the document from")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -101,7 +102,7 @@ public class DeleteElasticsearch5 extends AbstractElasticsearch5TransportClientP
             .displayName("Type")
             .description("The type of this document to be deleted")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/FetchElasticsearch5.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/FetchElasticsearch5.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/FetchElasticsearch5.java
index 9041038..4805c56 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/FetchElasticsearch5.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/FetchElasticsearch5.java
@@ -26,6 +26,7 @@ import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.logging.ComponentLog;
@@ -88,7 +89,7 @@ public class FetchElasticsearch5 extends AbstractElasticsearch5TransportClientPr
             .displayName("Document Identifier")
             .description("The identifier for the document to be fetched")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -97,7 +98,7 @@ public class FetchElasticsearch5 extends AbstractElasticsearch5TransportClientPr
             .displayName("Index")
             .description("The name of the index to read from")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -106,7 +107,7 @@ public class FetchElasticsearch5 extends AbstractElasticsearch5TransportClientPr
             .displayName("Type")
             .description("The type of this document (used by Elasticsearch for indexing and searching)")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearch5.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearch5.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearch5.java
index 1075592..2cfe01d 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearch5.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearch5.java
@@ -29,6 +29,7 @@ import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.ProcessContext;
@@ -91,7 +92,7 @@ public class PutElasticsearch5 extends AbstractElasticsearch5TransportClientProc
             .displayName("Identifier Attribute")
             .description("The name of the attribute containing the identifier for each FlowFile")
             .required(true)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .addValidator(StandardValidators.ATTRIBUTE_KEY_VALIDATOR)
             .build();
 
@@ -100,7 +101,7 @@ public class PutElasticsearch5 extends AbstractElasticsearch5TransportClientProc
             .displayName("Index")
             .description("The name of the index to insert into")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(NON_EMPTY_EL_VALIDATOR)
             .build();
 
@@ -109,7 +110,7 @@ public class PutElasticsearch5 extends AbstractElasticsearch5TransportClientProc
             .displayName("Type")
             .description("The type of this document (used by Elasticsearch for indexing and searching)")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(NON_EMPTY_EL_VALIDATOR)
             .build();
 
@@ -118,7 +119,7 @@ public class PutElasticsearch5 extends AbstractElasticsearch5TransportClientProc
             .displayName("Index Operation")
             .description("The type of the operation used to index (index, update, upsert)")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(NON_EMPTY_EL_VALIDATOR)
             .defaultValue("index")
             .build();
@@ -130,7 +131,7 @@ public class PutElasticsearch5 extends AbstractElasticsearch5TransportClientProc
             .required(true)
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
             .defaultValue("100")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     private static final Set<Relationship> relationships;

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/test/java/org/apache/nifi/processors/elasticsearch/ITDeleteElasticsearch5Test.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/test/java/org/apache/nifi/processors/elasticsearch/ITDeleteElasticsearch5Test.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/test/java/org/apache/nifi/processors/elasticsearch/ITDeleteElasticsearch5Test.java
index f9dfe58..0ce65a3 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/test/java/org/apache/nifi/processors/elasticsearch/ITDeleteElasticsearch5Test.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/test/java/org/apache/nifi/processors/elasticsearch/ITDeleteElasticsearch5Test.java
@@ -16,6 +16,10 @@
  */
 package org.apache.nifi.processors.elasticsearch;
 
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.HashMap;
+
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
@@ -26,10 +30,6 @@ import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.HashMap;
-
 /**
  * Integration test for delete processor. Please set the hosts, cluster name, index and type etc before running the integrations.
  */
@@ -59,7 +59,6 @@ public class ITDeleteElasticsearch5Test {
     @Test
     public void testPutAndDeleteIntegrationTestSuccess() {
         final TestRunner runnerPut = TestRunners.newTestRunner(new PutElasticsearch5());
-        runnerPut.setValidateExpressionUsage(false);
         runnerPut.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, clusterName);
         runnerPut.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
         runnerPut.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "5s");
@@ -82,7 +81,6 @@ public class ITDeleteElasticsearch5Test {
         runnerPut.assertAllFlowFilesTransferred(PutElasticsearch5.REL_SUCCESS, 1);
 
         final TestRunner runnerDelete = TestRunners.newTestRunner(new DeleteElasticsearch5());
-        runnerDelete.setValidateExpressionUsage(false);
 
         runnerDelete.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, clusterName);
         runnerDelete.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
@@ -108,7 +106,6 @@ public class ITDeleteElasticsearch5Test {
     @Test
     public void testDeleteIntegrationTestDocumentNotFound() {
         final TestRunner runnerDelete = TestRunners.newTestRunner(new DeleteElasticsearch5());
-        runnerDelete.setValidateExpressionUsage(false);
 
         runnerDelete.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, clusterName);
         runnerDelete.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
@@ -138,7 +135,6 @@ public class ITDeleteElasticsearch5Test {
     @Test
     public void testDeleteIntegrationTestBadIndex() {
         final TestRunner runnerDelete = TestRunners.newTestRunner(new DeleteElasticsearch5());
-        runnerDelete.setValidateExpressionUsage(false);
 
         runnerDelete.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, clusterName);
         runnerDelete.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
@@ -168,7 +164,6 @@ public class ITDeleteElasticsearch5Test {
     @Test
     public void testDeleteIntegrationTestBadType() {
         final TestRunner runnerDelete = TestRunners.newTestRunner(new DeleteElasticsearch5());
-        runnerDelete.setValidateExpressionUsage(false);
 
         runnerDelete.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, clusterName);
         runnerDelete.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestDeleteElasticsearch5.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestDeleteElasticsearch5.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestDeleteElasticsearch5.java
index 83e8a2d..a6c9811 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestDeleteElasticsearch5.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestDeleteElasticsearch5.java
@@ -16,6 +16,13 @@
  */
 package org.apache.nifi.processors.elasticsearch;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.concurrent.ExecutionException;
+
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
@@ -28,13 +35,6 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.concurrent.ExecutionException;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-
 public class TestDeleteElasticsearch5 {
 
     private String documentId;
@@ -71,7 +71,6 @@ public class TestDeleteElasticsearch5 {
 
         runner = TestRunners.newTestRunner(mockDeleteProcessor);
 
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch");
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "5s");
@@ -227,7 +226,6 @@ public class TestDeleteElasticsearch5 {
         };
         runner = TestRunners.newTestRunner(mockDeleteProcessor);
 
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch");
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "5s");
@@ -275,7 +273,6 @@ public class TestDeleteElasticsearch5 {
         };
         runner = TestRunners.newTestRunner(mockDeleteProcessor);
 
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch");
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "5s");

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestFetchElasticsearch5.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestFetchElasticsearch5.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestFetchElasticsearch5.java
index 26476d0..8605b09 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestFetchElasticsearch5.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestFetchElasticsearch5.java
@@ -83,7 +83,6 @@ public class TestFetchElasticsearch5 {
     @Test
     public void testFetchElasticsearch5OnTrigger() throws IOException {
         runner = TestRunners.newTestRunner(new FetchElasticsearch5TestProcessor(true)); // all docs are found
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch");
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "5s");
@@ -110,7 +109,6 @@ public class TestFetchElasticsearch5 {
     @Test
     public void testFetchElasticsearch5OnTriggerEL() throws IOException {
         runner = TestRunners.newTestRunner(new FetchElasticsearch5TestProcessor(true)); // all docs are found
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "${cluster.name}");
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "${hosts}");
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "${ping.timeout}");
@@ -147,7 +145,6 @@ public class TestFetchElasticsearch5 {
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.SAMPLER_INTERVAL, "5s");
         runner.setProperty(FetchElasticsearch5.INDEX, "doc");
         runner.setProperty(FetchElasticsearch5.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(FetchElasticsearch5.DOC_ID, "${doc_id}");
 
         runner.enqueue(docExample, new HashMap<String, String>() {{
@@ -171,7 +168,6 @@ public class TestFetchElasticsearch5 {
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.SAMPLER_INTERVAL, "5s");
         runner.setProperty(FetchElasticsearch5.INDEX, "doc");
         runner.setProperty(FetchElasticsearch5.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(FetchElasticsearch5.DOC_ID, "${doc_id}");
 
         runner.assertNotValid();
@@ -187,7 +183,6 @@ public class TestFetchElasticsearch5 {
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.SAMPLER_INTERVAL, "5s");
         runner.setProperty(FetchElasticsearch5.INDEX, "doc");
         runner.setProperty(FetchElasticsearch5.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(FetchElasticsearch5.DOC_ID, "${doc_id}");
 
         // No Node Available exception
@@ -273,7 +268,6 @@ public class TestFetchElasticsearch5 {
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.SAMPLER_INTERVAL, "5s");
         runner.setProperty(FetchElasticsearch5.INDEX, "doc");
         runner.setProperty(FetchElasticsearch5.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(FetchElasticsearch5.DOC_ID, "${doc_id}");
 
         // Allow time for the controller service to fully initialize
@@ -387,7 +381,6 @@ public class TestFetchElasticsearch5 {
         System.out.println("Starting test " + new Object() {
         }.getClass().getEnclosingMethod().getName());
         final TestRunner runner = TestRunners.newTestRunner(new FetchElasticsearch5());
-        runner.setValidateExpressionUsage(true);
 
         //Local Cluster - Mac pulled from brew
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch_brew");
@@ -418,7 +411,6 @@ public class TestFetchElasticsearch5 {
         System.out.println("Starting test " + new Object() {
         }.getClass().getEnclosingMethod().getName());
         final TestRunner runner = TestRunners.newTestRunner(new FetchElasticsearch5());
-        runner.setValidateExpressionUsage(true);
 
         //Local Cluster - Mac pulled from brew
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch_brew");

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestPutElasticsearch5.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestPutElasticsearch5.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestPutElasticsearch5.java
index 3e80731..eccbaf2 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestPutElasticsearch5.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestPutElasticsearch5.java
@@ -16,6 +16,22 @@
  */
 package org.apache.nifi.processors.elasticsearch;
 
+import static org.junit.Assert.assertNotNull;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
@@ -45,23 +61,6 @@ import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.InetSocketAddress;
-import java.net.MalformedURLException;
-import java.util.HashMap;
-import java.util.List;
-import java.util.concurrent.ExecutionException;
-
-import static org.junit.Assert.assertNotNull;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.when;
-
 public class TestPutElasticsearch5 {
 
     private InputStream docExample;
@@ -81,7 +80,6 @@ public class TestPutElasticsearch5 {
     @Test
     public void testPutElasticSearchOnTrigger() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearch5TestProcessor(false)); // no failures
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch");
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "5s");
@@ -109,7 +107,6 @@ public class TestPutElasticsearch5 {
     @Test
     public void testPutElasticSearchOnTriggerEL() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearch5TestProcessor(false)); // no failures
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "${cluster.name}");
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "${hosts}");
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "${ping.timeout}");
@@ -142,7 +139,6 @@ public class TestPutElasticsearch5 {
     @Test
     public void testPutElasticSearchOnTriggerBadDocIdentifier() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearch5TestProcessor(false)); // no failures
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch");
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "5s");
@@ -167,7 +163,6 @@ public class TestPutElasticsearch5 {
     @Test
     public void testPutElasticSearchOnTriggerWithFailures() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearch5TestProcessor(true)); // simulate failures
-        runner.setValidateExpressionUsage(false);
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch");
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "5s");
@@ -198,7 +193,6 @@ public class TestPutElasticsearch5 {
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.SAMPLER_INTERVAL, "5s");
         runner.setProperty(PutElasticsearch5.INDEX, "doc");
         runner.setProperty(PutElasticsearch5.TYPE, "status");
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(PutElasticsearch5.ID_ATTRIBUTE, "doc_id");
 
         // No Node Available exception
@@ -255,7 +249,6 @@ public class TestPutElasticsearch5 {
     @Test
     public void testPutElasticsearch5OnTriggerWithNoIdAttribute() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearch5TestProcessor(true)); // simulate failures
-        runner.setValidateExpressionUsage(false);
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch");
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "5s");
@@ -276,7 +269,6 @@ public class TestPutElasticsearch5 {
     @Test
     public void testPutElasticsearch5OnTriggerWithIndexFromAttribute() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearch5TestProcessor(false));
-        runner.setValidateExpressionUsage(false);
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch");
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "5s");
@@ -313,7 +305,6 @@ public class TestPutElasticsearch5 {
     @Test
     public void testPutElasticSearchOnTriggerWithInvalidIndexOp() throws IOException {
         runner = TestRunners.newTestRunner(new PutElasticsearch5TestProcessor(false)); // no failures
-        runner.setValidateExpressionUsage(true);
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch");
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "5s");
@@ -448,7 +439,6 @@ public class TestPutElasticsearch5 {
         System.out.println("Starting test " + new Object() {
         }.getClass().getEnclosingMethod().getName());
         final TestRunner runner = TestRunners.newTestRunner(new PutElasticsearch5());
-        runner.setValidateExpressionUsage(false);
 
         //Local Cluster - Mac pulled from brew
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch_brew");
@@ -480,7 +470,6 @@ public class TestPutElasticsearch5 {
         System.out.println("Starting test " + new Object() {
         }.getClass().getEnclosingMethod().getName());
         final TestRunner runner = TestRunners.newTestRunner(new PutElasticsearch5());
-        runner.setValidateExpressionUsage(false);
 
         //Local Cluster - Mac pulled from brew
         runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch_brew");

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchHttpProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchHttpProcessor.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchHttpProcessor.java
index faaf523..2ed6f79 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchHttpProcessor.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchHttpProcessor.java
@@ -26,6 +26,7 @@ import okhttp3.Response;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.exception.ProcessException;
@@ -62,7 +63,7 @@ public abstract class AbstractElasticsearchHttpProcessor extends AbstractElastic
             .description("Elasticsearch URL which will be connected to, including scheme (http, e.g.), host, and port. The default port for the REST API is 9200.")
             .required(true)
             .addValidator(StandardValidators.URL_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor PROXY_HOST = new PropertyDescriptor.Builder()
@@ -88,7 +89,7 @@ public abstract class AbstractElasticsearchHttpProcessor extends AbstractElastic
             .required(true)
             .defaultValue("5 secs")
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor RESPONSE_TIMEOUT = new PropertyDescriptor.Builder()
@@ -98,7 +99,7 @@ public abstract class AbstractElasticsearchHttpProcessor extends AbstractElastic
             .required(true)
             .defaultValue("15 secs")
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     private final AtomicReference<OkHttpClient> okHttpClientAtomicReference = new AtomicReference<>();
@@ -109,7 +110,7 @@ public abstract class AbstractElasticsearchHttpProcessor extends AbstractElastic
                 .name(propertyDescriptorName)
                 .required(false)
                 .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
                 .dynamic(true)
                 .build();
     }


[15/15] nifi git commit: NIFI-4149: Minor tweaks to verbiage

Posted by ma...@apache.org.
NIFI-4149: Minor tweaks to verbiage

This closes #2205.


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/644133dc
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/644133dc
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/644133dc

Branch: refs/heads/master
Commit: 644133dc353e8ca3b66658e768b1199c4820da06
Parents: 4c78779
Author: Mark Payne <ma...@hotmail.com>
Authored: Fri Apr 6 12:39:34 2018 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Fri Apr 6 12:42:38 2018 -0400

----------------------------------------------------------------------
 .../apache/nifi/web/api/dto/PropertyDescriptorDTO.java  | 12 ++++++------
 .../documentation/html/HtmlDocumentationWriter.java     |  8 ++++----
 .../html/ProcessorDocumentationWriterTest.java          |  2 +-
 .../java/org/apache/nifi/controller/TemplateUtils.java  |  2 +-
 .../java/org/apache/nifi/web/api/dto/DtoFactory.java    |  2 +-
 .../nifi-web-ui/src/main/webapp/js/nf/nf-common.js      |  2 +-
 6 files changed, 14 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/644133dc/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java
index e14b1bf..5834f15 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java
@@ -37,7 +37,7 @@ public class PropertyDescriptorDTO {
     private Boolean sensitive;
     private Boolean dynamic;
     private Boolean supportsEl;
-    private String scopeEl;
+    private String expressionLanguageScope;
     private String identifiesControllerService;
     private BundleDTO identifiesControllerServiceBundle;
 
@@ -171,14 +171,14 @@ public class PropertyDescriptorDTO {
      * @return specifies the scope of expression language evaluation
      */
     @ApiModelProperty(
-            value = "Scope of the EL evaluation for the property."
+            value = "Scope of the Expression Language evaluation for the property."
     )
-    public String getScopeEl() {
-        return scopeEl;
+    public String getExpressionLanguageScope() {
+        return expressionLanguageScope;
     }
 
-    public void setScopeEl(String scopeEl) {
-        this.scopeEl = scopeEl;
+    public void setExpressionLanguageScope(String scope) {
+        this.expressionLanguageScope = scope;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/nifi/blob/644133dc/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java
index 48fa109..fb2c5a0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java
@@ -525,8 +525,8 @@ public class HtmlDocumentationWriter implements DocumentationWriter {
                 if (property.isExpressionLanguageSupported()) {
                     xmlStreamWriter.writeEmptyElement("br");
                     String text = "Supports Expression Language: true";
-                    final String perFF = " (will be evaluated using flow file attributes and registry)";
-                    final String registry = " (will be evaluated using registry only)";
+                    final String perFF = " (will be evaluated using flow file attributes and variable registry)";
+                    final String registry = " (will be evaluated using variable registry only)";
                     final InputRequirement inputRequirement = configurableComponent.getClass().getAnnotation(InputRequirement.class);
 
                     switch(property.getExpressionLanguageScope()) {
@@ -627,10 +627,10 @@ public class HtmlDocumentationWriter implements DocumentationWriter {
                 } else {
                     switch(dynamicProperty.expressionLanguageScope()) {
                         case FLOWFILE_ATTRIBUTES:
-                            text = "Supports Expression Language: true (will be evaluated using flow file attributes and registry)";
+                            text = "Supports Expression Language: true (will be evaluated using flow file attributes and variable registry)";
                             break;
                         case VARIABLE_REGISTRY:
-                            text = "Supports Expression Language: true (will be evaluated using registry only)";
+                            text = "Supports Expression Language: true (will be evaluated using variable registry only)";
                             break;
                         case NONE:
                         default:

http://git-wip-us.apache.org/repos/asf/nifi/blob/644133dc/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java
index a047fd5..4e45b7b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java
@@ -87,7 +87,7 @@ public class ProcessorDocumentationWriterTest {
         assertNotContains(results, "Additional Details...");
 
         // check expression language scope
-        assertContains(results, "Supports Expression Language: true (will be evaluated using registry only)");
+        assertContains(results, "Supports Expression Language: true (will be evaluated using variable registry only)");
         assertContains(results, "Supports Expression Language: true (undefined scope)");
 
         // verify dynamic properties

http://git-wip-us.apache.org/repos/asf/nifi/blob/644133dc/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateUtils.java
index 5b40f72..57ea19f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateUtils.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateUtils.java
@@ -209,7 +209,7 @@ public class TemplateUtils {
         descriptor.setRequired(null);
         descriptor.setSensitive(null);
         descriptor.setSupportsEl(null);
-        descriptor.setScopeEl(null);
+        descriptor.setExpressionLanguageScope(null);
         descriptor.setIdentifiesControllerServiceBundle(null);
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/644133dc/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
index d0bd539..3aac227 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
@@ -3605,7 +3605,7 @@ public final class DtoFactory {
         String description = propertyDescriptor.isExpressionLanguageSupported()
                 && propertyDescriptor.getExpressionLanguageScope().equals(ExpressionLanguageScope.NONE)
                 ? "true (undefined scope)" : propertyDescriptor.getExpressionLanguageScope().getDescription();
-        dto.setScopeEl(description);
+        dto.setExpressionLanguageScope(description);
 
         // set the identifies controller service is applicable
         if (propertyDescriptor.getControllerServiceDefinition() != null) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/644133dc/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js
index 238c8be..99b2d35 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js
@@ -900,7 +900,7 @@
                     tipContent.push('<b>Default value:</b> ' + nfCommon.escapeHtml(propertyDescriptor.defaultValue));
                 }
                 if (!nfCommon.isBlank(propertyDescriptor.supportsEl)) {
-                    tipContent.push('<b>Expression language scope:</b> ' + nfCommon.escapeHtml(propertyDescriptor.scopeEl));
+                    tipContent.push('<b>Expression language scope:</b> ' + nfCommon.escapeHtml(propertyDescriptor.expressionLanguageScope));
                 }
                 if (!nfCommon.isBlank(propertyDescriptor.identifiesControllerService)) {
                     var formattedType = nfCommon.formatType({


[04/15] nifi git commit: NIFI-4149 - Indicate if EL is evaluated against FFs or not - take into account input requirement for documentation rendering - Renamed variable registry scope and added comments - Doc + change in mock framework to check scope + u

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LogMessage.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LogMessage.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LogMessage.java
index 2d6b318..d8e33ca 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LogMessage.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LogMessage.java
@@ -24,6 +24,7 @@ import org.apache.nifi.annotation.behavior.SupportsBatching;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -56,7 +57,7 @@ public class LogMessage extends AbstractProcessor {
             .description("The Log Level to use when logging the message")
             .allowableValues(MessageLogLevel.values())
             .defaultValue(MessageLogLevel.info.toString())
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor LOG_PREFIX = new PropertyDescriptor.Builder()
@@ -66,7 +67,7 @@ public class LogMessage extends AbstractProcessor {
             .description("Log prefix appended to the log lines. " +
                     "It helps to distinguish the output of multiple LogMessage processors.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor LOG_MESSAGE = new PropertyDescriptor.Builder()
@@ -75,7 +76,7 @@ public class LogMessage extends AbstractProcessor {
             .required(false)
             .description("The log message to emit")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final Relationship REL_SUCCESS = new Relationship.Builder()
@@ -120,7 +121,12 @@ public class LogMessage extends AbstractProcessor {
     @Override
     public void onTrigger(final ProcessContext context, final ProcessSession session) {
 
-        final String logLevelValue = context.getProperty(LOG_LEVEL).getValue().toLowerCase();
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final String logLevelValue = context.getProperty(LOG_LEVEL).evaluateAttributeExpressions(flowFile).getValue().toLowerCase();
 
         final MessageLogLevel logLevel;
         try {
@@ -154,11 +160,6 @@ public class LogMessage extends AbstractProcessor {
             return;
         }
 
-        final FlowFile flowFile = session.get();
-        if (flowFile == null) {
-            return;
-        }
-
         processFlowFile(logger, logLevel, flowFile, context);
         session.transfer(flowFile, REL_SUCCESS);
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LookupAttribute.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LookupAttribute.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LookupAttribute.java
index 64a83e9..56ad58f 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LookupAttribute.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LookupAttribute.java
@@ -44,6 +44,7 @@ import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.lookup.LookupFailureException;
@@ -65,7 +66,7 @@ import org.apache.nifi.processor.util.StandardValidators;
 @CapabilityDescription("Lookup attributes from a lookup service")
 @DynamicProperty(name = "The name of the attribute to add to the FlowFile",
     value = "The name of the key or property to retrieve from the lookup service",
-    supportsExpressionLanguage = true,
+    expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
     description = "Adds a FlowFile attribute specified by the dynamic property's key with the value found in the lookup service using the the dynamic property's value")
 public class LookupAttribute extends AbstractProcessor {
 
@@ -150,7 +151,7 @@ public class LookupAttribute extends AbstractProcessor {
             .required(false)
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
             .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .dynamic(true)
             .build();
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LookupRecord.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LookupRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LookupRecord.java
index 6d7db86..589272f 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LookupRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LookupRecord.java
@@ -45,6 +45,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.lookup.LookupService;
 import org.apache.nifi.processor.ProcessContext;
@@ -81,8 +82,8 @@ import org.apache.nifi.util.Tuple;
     + "that match will be updated. If there is no match in the configured LookupService, then no fields will be updated. I.e., it will not overwrite an existing value in the Record "
     + "with a null value. Please note, however, that if the results returned by the LookupService are not accounted for in your schema (specifically, "
     + "the schema that is configured for your Record Writer) then the fields will not be written out to the FlowFile.")
-@DynamicProperty(name = "Value To Lookup", value = "Valid Record Path", supportsExpressionLanguage = true, description = "A RecordPath that points to the field whose value will be "
-    + "looked up in the configured Lookup Service")
+@DynamicProperty(name = "Value To Lookup", value = "Valid Record Path", expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
+                    description = "A RecordPath that points to the field whose value will be looked up in the configured Lookup Service")
 @SeeAlso(value = {ConvertRecord.class, SplitRecord.class}, classNames = {"org.apache.nifi.lookup.SimpleKeyValueLookupService", "org.apache.nifi.lookup.maxmind.IPLookupService"})
 public class LookupRecord extends AbstractRouteRecord<Tuple<Map<String, RecordPath>, RecordPath>> {
 
@@ -115,7 +116,7 @@ public class LookupRecord extends AbstractRouteRecord<Tuple<Map<String, RecordPa
             + "If not specified, the value that is returned from the Lookup Service will be ignored, except for determining whether the FlowFile should "
             + "be routed to the 'matched' or 'unmatched' Relationship.")
         .addValidator(new RecordPathValidator())
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .required(false)
         .build();
 
@@ -133,7 +134,7 @@ public class LookupRecord extends AbstractRouteRecord<Tuple<Map<String, RecordPa
         .name("routing-strategy")
         .displayName("Routing Strategy")
         .description("Specifies how to route records after a Lookup has completed")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues(ROUTE_TO_SUCCESS, ROUTE_TO_MATCHED_UNMATCHED)
         .defaultValue(ROUTE_TO_SUCCESS.getValue())
         .required(true)
@@ -186,7 +187,7 @@ public class LookupRecord extends AbstractRouteRecord<Tuple<Map<String, RecordPa
             .name(propertyDescriptorName)
             .description("A RecordPath that points to the field whose value will be looked up in the configured Lookup Service")
             .addValidator(new RecordPathValidator())
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(false)
             .dynamic(true)
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
index cc9f6f7..6978c9d 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
@@ -70,9 +70,9 @@ import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.flowfile.attributes.FragmentAttributes;
@@ -265,7 +265,7 @@ public class MergeContent extends BinFiles {
             .description("If specified, like FlowFiles will be binned together, where 'like FlowFiles' means FlowFiles that have the same value for "
                     + "this Attribute. If not specified, FlowFiles are bundled by the order in which they are pulled from the queue.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.ATTRIBUTE_KEY_VALIDATOR)
             .defaultValue(null)
             .build();
@@ -285,7 +285,7 @@ public class MergeContent extends BinFiles {
                     + "binary-concatenation merge strategy; otherwise, it is ignored.")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     public static final PropertyDescriptor FOOTER = new PropertyDescriptor.Builder()
             .name("Footer File")
@@ -294,7 +294,7 @@ public class MergeContent extends BinFiles {
                     + "binary-concatenation merge strategy; otherwise, it is ignored.")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     public static final PropertyDescriptor DEMARCATOR = new PropertyDescriptor.Builder()
             .name("Demarcator File")
@@ -303,7 +303,7 @@ public class MergeContent extends BinFiles {
                     + "using the binary-concatenation merge strategy; otherwise, it is ignored.")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     public static final PropertyDescriptor COMPRESSION_LEVEL = new PropertyDescriptor.Builder()
             .name("Compression Level")
@@ -327,7 +327,7 @@ public class MergeContent extends BinFiles {
                     + "(e.g. ${file.lastModifiedTime} or static value, both of which must match the ISO8601 format 'yyyy-MM-dd'T'HH:mm:ssZ'; if using "
                     + "other merge strategy or left blank, this value is ignored")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .defaultValue("${file.lastModifiedTime}")
             .build();
@@ -656,13 +656,13 @@ public class MergeContent extends BinFiles {
         private byte[] getDelimiterFileContent(final ProcessContext context, final List<FlowFile> flowFiles, final PropertyDescriptor descriptor)
                 throws IOException {
             byte[] property = null;
-            final String descriptorValue = context.getProperty(descriptor).evaluateAttributeExpressions().getValue();
-            if (descriptorValue != null && flowFiles != null && flowFiles.size() > 0) {
-                final String content = new String(readContent(descriptorValue), StandardCharsets.UTF_8);
+            if (flowFiles != null && flowFiles.size() > 0) {
                 final FlowFile flowFile = flowFiles.get(0);
-                if (flowFile != null && content != null) {
-                    final PropertyValue propVal = context.newPropertyValue(content).evaluateAttributeExpressions(flowFile);
-                    property = propVal.getValue().getBytes(StandardCharsets.UTF_8);
+                if (flowFile != null) {
+                    final String value = context.getProperty(descriptor).evaluateAttributeExpressions(flowFile).getValue();
+                    if (value != null) {
+                        property = readContent(value);
+                    }
                 }
             }
             return property;

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeRecord.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeRecord.java
index b0e3f48..3b27529 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeRecord.java
@@ -43,6 +43,7 @@ import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.avro.AvroTypeUtil;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.FragmentAttributes;
 import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
@@ -145,7 +146,7 @@ public class MergeRecord extends AbstractSessionFactoryProcessor {
         .description("If specified, two FlowFiles will be binned together only if they have the same value for "
             + "this Attribute. If not specified, FlowFiles are bundled by the order in which they are pulled from the queue.")
         .required(false)
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .addValidator(StandardValidators.ATTRIBUTE_KEY_VALIDATOR)
         .defaultValue(null)
         .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyBytes.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyBytes.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyBytes.java
index 6d8bee3..a2c8efe 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyBytes.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyBytes.java
@@ -33,6 +33,7 @@ import org.apache.nifi.annotation.behavior.SideEffectFree;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -66,7 +67,7 @@ public class ModifyBytes extends AbstractProcessor {
             .required(true)
             .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
             .defaultValue("0 B")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     public static final PropertyDescriptor END_OFFSET = new PropertyDescriptor.Builder()
             .name("End Offset")
@@ -74,7 +75,7 @@ public class ModifyBytes extends AbstractProcessor {
             .required(true)
             .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
             .defaultValue("0 B")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     public static final PropertyDescriptor REMOVE_ALL = new PropertyDescriptor.Builder()
             .name("Remove All Content")

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MonitorActivity.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MonitorActivity.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MonitorActivity.java
index 201d287..c7efc1f 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MonitorActivity.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MonitorActivity.java
@@ -48,6 +48,7 @@ import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.components.state.StateManager;
 import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.logging.ComponentLog;
@@ -98,7 +99,7 @@ public class MonitorActivity extends AbstractProcessor {
             .name("Activity Restored Message")
             .description("The message that will be the content of FlowFiles that are sent to 'activity.restored' relationship")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .defaultValue("Activity restored at time: ${now():format('yyyy/MM/dd HH:mm:ss')} after being inactive for ${inactivityDurationMillis:toNumber():divide(60000)} minutes")
             .build();
@@ -106,7 +107,7 @@ public class MonitorActivity extends AbstractProcessor {
             .name("Inactivity Message")
             .description("The message that will be the content of FlowFiles that are sent to the 'inactive' relationship")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .defaultValue("Lacking activity as of time: ${now():format('yyyy/MM/dd HH:mm:ss')}; flow has been inactive for ${inactivityDurationMillis:toNumber():divide(60000)} minutes")
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Notify.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Notify.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Notify.java
index 4493390..a8d3000 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Notify.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Notify.java
@@ -38,6 +38,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.distributed.cache.client.AtomicDistributedMapCacheClient;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -79,7 +80,7 @@ public class Notify extends AbstractProcessor {
                 "be evaluated against a FlowFile in order to determine the release signal cache key")
             .required(true)
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(ResultType.STRING, true))
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor SIGNAL_COUNTER_NAME = new PropertyDescriptor.Builder()
@@ -91,7 +92,7 @@ public class Notify extends AbstractProcessor {
                 "of different types of events, such as success or failure, or destination data source names, etc.")
             .required(true)
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(ResultType.STRING, true))
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue(WaitNotifyProtocol.DEFAULT_COUNT_NAME)
             .build();
 
@@ -108,7 +109,7 @@ public class Notify extends AbstractProcessor {
                 "One (1) can open a corresponding Wait processor, and Zero (0) can negate it as if closing a gate.")
             .required(true)
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(ResultType.STRING, true))
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("1")
             .build();
 
@@ -132,7 +133,7 @@ public class Notify extends AbstractProcessor {
                     + "uuid attribute will not be cached regardless of this value.  If blank, no attributes "
                     + "will be cached.")
             .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
 
     public static final Relationship REL_SUCCESS = new Relationship.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PartitionRecord.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PartitionRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PartitionRecord.java
index 4a86e81..43e1e4b 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PartitionRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PartitionRecord.java
@@ -45,6 +45,7 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -79,7 +80,7 @@ import org.apache.nifi.serialization.record.util.DataTypeUtils;
         + "for a Record, an attribute is added to the outgoing FlowFile. The name of the attribute is the same as the name of this property. The value of the attribute is the same as "
         + "the value of the field in the Record that the RecordPath points to. Note that no attribute will be added if the value returned for the RecordPath is null or is not a scalar "
         + "value (i.e., the value is an Array, Map, or Record).",
-    supportsExpressionLanguage=true)
+    expressionLanguageScope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
 @WritesAttributes({
     @WritesAttribute(attribute="record.count", description="The number of records in an outgoing FlowFile"),
     @WritesAttribute(attribute="mime.type", description="The MIME Type that the configured Record Writer indicates is appropriate"),
@@ -160,7 +161,7 @@ public class PartitionRecord extends AbstractProcessor {
             .name(propertyDescriptorName)
             .dynamic(true)
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(new RecordPathValidator())
             .build();
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java
index 00e51d2..8bb24bf 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java
@@ -61,6 +61,7 @@ import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.logging.ComponentLog;
@@ -159,7 +160,7 @@ public class PostHTTP extends AbstractProcessor {
             .required(true)
             .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("https?\\://.*")))
             .addValidator(StandardValidators.URL_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     public static final PropertyDescriptor SEND_AS_FLOWFILE = new PropertyDescriptor.Builder()
             .name("Send as FlowFile")
@@ -259,7 +260,7 @@ public class PostHTTP extends AbstractProcessor {
             .description("The Content-Type to specify for the content of the FlowFile being POSTed if " + SEND_AS_FLOWFILE.getName() + " is false. "
                     + "In the case of an empty value after evaluating an expression language expression, Content-Type defaults to " + DEFAULT_CONTENT_TYPE)
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("${" + CoreAttributes.MIME_TYPE.key() + "}")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDatabaseRecord.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDatabaseRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDatabaseRecord.java
index c6f7766..9b236fe 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDatabaseRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDatabaseRecord.java
@@ -29,6 +29,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.dbcp.DBCPService;
 import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
@@ -168,7 +169,7 @@ public class PutDatabaseRecord extends AbstractSessionFactoryProcessor {
             .displayName("Catalog Name")
             .description("The name of the catalog that the statement should update. This may not apply for the database that you are updating. In this case, leave the field empty")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -177,7 +178,7 @@ public class PutDatabaseRecord extends AbstractSessionFactoryProcessor {
             .displayName("Schema Name")
             .description("The name of the schema that the table belongs to. This may not apply for the database that you are updating. In this case, leave the field empty")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -186,7 +187,7 @@ public class PutDatabaseRecord extends AbstractSessionFactoryProcessor {
             .displayName("Table Name")
             .description("The name of the table that the statement should affect.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -224,7 +225,7 @@ public class PutDatabaseRecord extends AbstractSessionFactoryProcessor {
                     + "This property is ignored if the Statement Type is INSERT")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     static final PropertyDescriptor FIELD_CONTAINING_SQL = new PropertyDescriptor.Builder()
@@ -234,7 +235,7 @@ public class PutDatabaseRecord extends AbstractSessionFactoryProcessor {
                     + "of the field must be a single SQL statement. If the Statement Type is not 'SQL', this field is ignored.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     static final PropertyDescriptor QUOTED_IDENTIFIERS = new PropertyDescriptor.Builder()
@@ -261,7 +262,7 @@ public class PutDatabaseRecord extends AbstractSessionFactoryProcessor {
             .defaultValue("0 seconds")
             .required(true)
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     protected static List<PropertyDescriptor> propDescriptors;
@@ -323,7 +324,7 @@ public class PutDatabaseRecord extends AbstractSessionFactoryProcessor {
                 .required(false)
                 .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
                 .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
                 .dynamic(true)
                 .build();
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDistributedMapCache.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDistributedMapCache.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDistributedMapCache.java
index 180ff8e..644796d 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDistributedMapCache.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDistributedMapCache.java
@@ -43,6 +43,7 @@ import org.apache.nifi.distributed.cache.client.Serializer;
 import org.apache.nifi.distributed.cache.client.exception.DeserializationException;
 import org.apache.nifi.distributed.cache.client.exception.SerializationException;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -83,7 +84,7 @@ public class PutDistributedMapCache extends AbstractProcessor {
             "be evaluated against a FlowFile in order to determine the cache key")
         .required(true)
         .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(ResultType.STRING, true))
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .build();
 
     public static final AllowableValue CACHE_UPDATE_REPLACE = new AllowableValue("replace", "Replace if present",
@@ -106,7 +107,7 @@ public class PutDistributedMapCache extends AbstractProcessor {
         .required(false)
         .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
         .defaultValue("1 MB")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .build();
 
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutEmail.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutEmail.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutEmail.java
index aedf947..fcf017e 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutEmail.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutEmail.java
@@ -58,6 +58,7 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.logging.ComponentLog;
@@ -83,7 +84,7 @@ public class PutEmail extends AbstractProcessor {
             .name("SMTP Hostname")
             .description("The hostname of the SMTP host")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
     public static final PropertyDescriptor SMTP_PORT = new PropertyDescriptor.Builder()
@@ -91,20 +92,20 @@ public class PutEmail extends AbstractProcessor {
             .description("The Port used for SMTP communications")
             .required(true)
             .defaultValue("25")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.PORT_VALIDATOR)
             .build();
     public static final PropertyDescriptor SMTP_USERNAME = new PropertyDescriptor.Builder()
             .name("SMTP Username")
             .description("Username for the SMTP account")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .required(false)
             .build();
     public static final PropertyDescriptor SMTP_PASSWORD = new PropertyDescriptor.Builder()
             .name("SMTP Password")
             .description("Password for the SMTP account")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .required(false)
             .sensitive(true)
@@ -113,7 +114,7 @@ public class PutEmail extends AbstractProcessor {
             .name("SMTP Auth")
             .description("Flag indicating whether authentication should be used")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
             .defaultValue("true")
             .build();
@@ -121,7 +122,7 @@ public class PutEmail extends AbstractProcessor {
             .name("SMTP TLS")
             .description("Flag indicating whether TLS should be enabled")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
             .defaultValue("false")
             .build();
@@ -129,7 +130,7 @@ public class PutEmail extends AbstractProcessor {
             .name("SMTP Socket Factory")
             .description("Socket Factory to use for SMTP Connection")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .defaultValue("javax.net.ssl.SSLSocketFactory")
             .build();
@@ -137,7 +138,7 @@ public class PutEmail extends AbstractProcessor {
             .name("SMTP X-Mailer Header")
             .description("X-Mailer used in the header of the outgoing email")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .defaultValue("NiFi")
             .build();
@@ -145,7 +146,7 @@ public class PutEmail extends AbstractProcessor {
             .name("Content Type")
             .description("Mime Type used to interpret the contents of the email, such as text/plain or text/html")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .defaultValue("text/plain")
             .build();
@@ -153,35 +154,35 @@ public class PutEmail extends AbstractProcessor {
             .name("From")
             .description("Specifies the Email address to use as the sender")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
     public static final PropertyDescriptor TO = new PropertyDescriptor.Builder()
             .name("To")
             .description("The recipients to include in the To-Line of the email")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
     public static final PropertyDescriptor CC = new PropertyDescriptor.Builder()
             .name("CC")
             .description("The recipients to include in the CC-Line of the email")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
     public static final PropertyDescriptor BCC = new PropertyDescriptor.Builder()
             .name("BCC")
             .description("The recipients to include in the BCC-Line of the email")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
     public static final PropertyDescriptor SUBJECT = new PropertyDescriptor.Builder()
             .name("Subject")
             .description("The email subject")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("Message from NiFi")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
@@ -189,7 +190,7 @@ public class PutEmail extends AbstractProcessor {
             .name("Message")
             .description("The body of the email message")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
     public static final PropertyDescriptor ATTACH_FILE = new PropertyDescriptor.Builder()
@@ -204,7 +205,7 @@ public class PutEmail extends AbstractProcessor {
             .displayName("Flow file content as message")
             .description("Specifies whether or not the FlowFile content should be the message of the email. If true, the 'Message' property is ignored.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
             .defaultValue("false")
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java
index 1f53ad1..0c95c75 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java
@@ -28,6 +28,7 @@ import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.logging.ComponentLog;
@@ -86,7 +87,7 @@ public class PutFile extends AbstractProcessor {
             .description("The directory to which files should be written. You may use expression language such as /aa/bb/${path}")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     public static final PropertyDescriptor MAX_DESTINATION_FILES = new PropertyDescriptor.Builder()
             .name("Maximum File Count")
@@ -107,7 +108,7 @@ public class PutFile extends AbstractProcessor {
                     + "You may also use expression language such as ${file.lastModifiedTime}.")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     public static final PropertyDescriptor CHANGE_PERMISSIONS = new PropertyDescriptor.Builder()
             .name("Permissions")
@@ -116,7 +117,7 @@ public class PutFile extends AbstractProcessor {
                     + "${file.permissions}.")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     public static final PropertyDescriptor CHANGE_OWNER = new PropertyDescriptor.Builder()
             .name("Owner")
@@ -124,7 +125,7 @@ public class PutFile extends AbstractProcessor {
                     + "${file.owner}.")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     public static final PropertyDescriptor CHANGE_GROUP = new PropertyDescriptor.Builder()
             .name("Group")
@@ -132,7 +133,7 @@ public class PutFile extends AbstractProcessor {
                     + "as ${file.group}.")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     public static final PropertyDescriptor CREATE_DIRS = new PropertyDescriptor.Builder()
             .name("Create Missing Directories")

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSQL.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSQL.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSQL.java
index fd9501b..55e4c5f 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSQL.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSQL.java
@@ -29,6 +29,7 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.dbcp.DBCPService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.FragmentAttributes;
 import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
@@ -129,7 +130,7 @@ public class PutSQL extends AbstractSessionFactoryProcessor {
                     + "to contain a valid SQL statement, to be issued by the processor to the database.")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     static final PropertyDescriptor SUPPORT_TRANSACTIONS = new PropertyDescriptor.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSyslog.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSyslog.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSyslog.java
index 81b016f..12b96f0 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSyslog.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSyslog.java
@@ -26,6 +26,7 @@ import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.ProcessContext;
@@ -111,13 +112,13 @@ public class PutSyslog extends AbstractSyslogProcessor {
             .description("The priority for the Syslog messages, excluding < >.")
             .required(true)
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     public static final PropertyDescriptor MSG_VERSION = new PropertyDescriptor
             .Builder().name("Message Version")
             .description("The version for the Syslog messages.")
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     public static final PropertyDescriptor MSG_TIMESTAMP = new PropertyDescriptor
             .Builder().name("Message Timestamp")
@@ -127,7 +128,7 @@ public class PutSyslog extends AbstractSyslogProcessor {
             .required(true)
             .defaultValue("${now():format('MMM d HH:mm:ss')}")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     public static final PropertyDescriptor MSG_HOSTNAME = new PropertyDescriptor
             .Builder().name("Message Hostname")
@@ -135,14 +136,14 @@ public class PutSyslog extends AbstractSyslogProcessor {
             .required(true)
             .defaultValue("${hostname(true)}")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     public static final PropertyDescriptor MSG_BODY = new PropertyDescriptor
             .Builder().name("Message Body")
             .description("The body for the Syslog messages.")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
             .name("SSL Context Service")

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryDatabaseTable.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryDatabaseTable.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryDatabaseTable.java
index aa81c34..fe1e4ef 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryDatabaseTable.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryDatabaseTable.java
@@ -34,6 +34,7 @@ import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.components.state.StateManager;
 import org.apache.nifi.components.state.StateMap;
 import org.apache.nifi.dbcp.DBCPService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.logging.ComponentLog;
@@ -105,8 +106,8 @@ import static org.apache.nifi.processors.standard.util.JdbcCommon.USE_AVRO_LOGIC
                 + "FlowFiles were produced"),
         @WritesAttribute(attribute = "maxvalue.*", description = "Each attribute contains the observed maximum value of a specified 'Maximum-value Column'. The "
                 + "suffix of the attribute is the name of the column. If Output Batch Size is set, then this attribute will not be populated.")})
-@DynamicProperty(name = "Initial Max Value", value = "Attribute Expression Language", supportsExpressionLanguage = false, description = "Specifies an initial "
-        + "max value for max value columns. Properties should be added in the format `initial.maxvalue.{max_value_column}`.")
+@DynamicProperty(name = "Initial Max Value", value = "Attribute Expression Language", expressionLanguageScope = ExpressionLanguageScope.NONE,
+                description = "Specifies an initial max value for max value columns. Properties should be added in the format `initial.maxvalue.{max_value_column}`.")
 public class QueryDatabaseTable extends AbstractDatabaseFetchProcessor {
 
     public static final String RESULT_TABLENAME = "tablename";
@@ -119,7 +120,7 @@ public class QueryDatabaseTable extends AbstractDatabaseFetchProcessor {
             .defaultValue("0")
             .required(true)
             .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor MAX_ROWS_PER_FLOW_FILE = new PropertyDescriptor.Builder()
@@ -130,7 +131,7 @@ public class QueryDatabaseTable extends AbstractDatabaseFetchProcessor {
             .defaultValue("0")
             .required(true)
             .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor OUTPUT_BATCH_SIZE = new PropertyDescriptor.Builder()
@@ -144,7 +145,7 @@ public class QueryDatabaseTable extends AbstractDatabaseFetchProcessor {
             .defaultValue("0")
             .required(true)
             .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor MAX_FRAGMENTS = new PropertyDescriptor.Builder()
@@ -156,7 +157,7 @@ public class QueryDatabaseTable extends AbstractDatabaseFetchProcessor {
             .defaultValue("0")
             .required(true)
             .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public QueryDatabaseTable() {

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryRecord.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryRecord.java
index 50e5dd7..2412736 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryRecord.java
@@ -64,6 +64,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -103,9 +104,11 @@ import org.apache.nifi.util.StopWatch;
     + "set of columns and aggregations. As a result, though, the schema that is derived will have no schema name, so it is important that the configured Record Writer not attempt "
     + "to write the Schema Name as an attribute if inheriting the Schema from the Record. See the Processor Usage documentation for more information.")
 @DynamicRelationship(name="<Property Name>", description="Each user-defined property defines a new Relationship for this Processor.")
-@DynamicProperty(name = "The name of the relationship to route data to", value="A SQL SELECT statement that is used to determine what data should be routed to this "
-        + "relationship.", supportsExpressionLanguage=true, description="Each user-defined property specifies a SQL SELECT statement to run over the data, with the data "
-        + "that is selected being routed to the relationship whose name is the property name")
+@DynamicProperty(name = "The name of the relationship to route data to",
+                 value="A SQL SELECT statement that is used to determine what data should be routed to this relationship.",
+                 expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
+                 description="Each user-defined property specifies a SQL SELECT statement to run over the data, with the data "
+                         + "that is selected being routed to the relationship whose name is the property name")
 @WritesAttributes({
     @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
     @WritesAttribute(attribute = "record.count", description = "The number of records selected by the query")
@@ -130,7 +133,7 @@ public class QueryRecord extends AbstractProcessor {
         .displayName("Include Zero Record FlowFiles")
         .description("When running the SQL statement against an incoming FlowFile, if the result has no data, "
             + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues("true", "false")
         .defaultValue("true")
         .required(true)
@@ -142,7 +145,7 @@ public class QueryRecord extends AbstractProcessor {
             + "the Processor will cache these values so that the Processor is much more efficient and much faster. However, if this is done, "
             + "then the schema that is derived for the first FlowFile processed must apply to all FlowFiles. If all FlowFiles will not have the exact "
             + "same schema, or if the SQL SELECT statement uses the Expression Language, this value should be set to false.")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues("true", "false")
         .defaultValue("true")
         .required(true)
@@ -238,7 +241,7 @@ public class QueryRecord extends AbstractProcessor {
                 + "SQL SELECT should select from the FLOWFILE table")
             .required(false)
             .dynamic(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(new SqlValidator())
             .build();
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java
index 6f37786..6cc9197 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java
@@ -32,6 +32,7 @@ import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
 import org.apache.nifi.expression.AttributeValueDecorator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -117,7 +118,7 @@ public class ReplaceText extends AbstractProcessor {
         .description("The Search Value to search for in the FlowFile content. Only used for 'Literal Replace' and 'Regex Replace' matching strategies")
         .required(true)
         .addValidator(Validator.VALID)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .defaultValue(DEFAULT_REGEX)
         .build();
     public static final PropertyDescriptor REPLACEMENT_VALUE = new PropertyDescriptor.Builder()
@@ -129,7 +130,7 @@ public class ReplaceText extends AbstractProcessor {
         .required(true)
         .defaultValue(DEFAULT_REPLACEMENT_VALUE)
         .addValidator(Validator.VALID)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .build();
     public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder()
         .name("Character Set")

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceTextWithMapping.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceTextWithMapping.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceTextWithMapping.java
index 84aeb75..429f640 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceTextWithMapping.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceTextWithMapping.java
@@ -51,6 +51,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.expression.AttributeValueDecorator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -79,7 +80,7 @@ public class ReplaceTextWithMapping extends AbstractProcessor {
             .description("The Regular Expression to search for in the FlowFile content")
             .required(true)
             .addValidator(StandardValidators.createRegexValidator(0, Integer.MAX_VALUE, true))
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("\\S+")
             .build();
     public static final PropertyDescriptor MATCHING_GROUP_FOR_LOOKUP_KEY = new PropertyDescriptor.Builder()
@@ -87,7 +88,7 @@ public class ReplaceTextWithMapping extends AbstractProcessor {
             .description("The number of the matching group of the provided regex to replace with the corresponding value from the mapping file (if it exists).")
             .addValidator(StandardValidators.INTEGER_VALIDATOR)
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue("0")
             .build();
     public static final PropertyDescriptor MAPPING_FILE = new PropertyDescriptor.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java
index c0e0a2a..6036c2a 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java
@@ -41,6 +41,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
@@ -66,8 +67,9 @@ import org.apache.nifi.processor.util.StandardValidators;
 @InputRequirement(Requirement.INPUT_REQUIRED)
 @Tags({"attributes", "routing", "Attribute Expression Language", "regexp", "regex", "Regular Expression", "Expression Language"})
 @CapabilityDescription("Routes FlowFiles based on their Attributes using the Attribute Expression Language")
-@DynamicProperty(name = "Relationship Name", value = "Attribute Expression Language", supportsExpressionLanguage = true, description = "Routes FlowFiles whose "
-        + "attributes match the Attribute Expression Language specified in the Dynamic Property Value to the Relationship specified in the Dynamic Property Key")
+@DynamicProperty(name = "Relationship Name", value = "Attribute Expression Language",
+                 expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES, description = "Routes FlowFiles whose attributes match the "
+                         + "Attribute Expression Language specified in the Dynamic Property Value to the Relationship specified in the Dynamic Property Key")
 @DynamicRelationship(name = "Name from Dynamic Property", description = "FlowFiles that match the Dynamic Property's Attribute Expression Language")
 @WritesAttributes({
     @WritesAttribute(attribute = RouteOnAttribute.ROUTE_ATTRIBUTE_KEY, description = "The relation to which the FlowFile was routed")
@@ -146,7 +148,7 @@ public class RouteOnAttribute extends AbstractProcessor {
                 .name(propertyDescriptorName)
                 .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(ResultType.BOOLEAN, false))
                 .dynamic(true)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
                 .build();
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java
index 5e0b892..e8e7ee1 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java
@@ -41,6 +41,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.expression.AttributeValueDecorator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -63,8 +64,9 @@ import org.apache.nifi.stream.io.StreamUtils;
         + "of the property is the name of the relationship and the value is a Regular Expression to match against the FlowFile "
         + "content. User-Defined properties do support the Attribute Expression Language, but the results are interpreted as "
         + "literal values, not Regular Expressions")
-@DynamicProperty(name = "Relationship Name", value = "A Regular Expression", supportsExpressionLanguage = true, description = "Routes FlowFiles whose "
-        + "content matches the regular expression defined by Dynamic Property's value to the Relationship defined by the Dynamic Property's key")
+@DynamicProperty(name = "Relationship Name", value = "A Regular Expression", expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
+                 description = "Routes FlowFiles whose content matches the regular expression defined by Dynamic Property's value to the "
+                         + "Relationship defined by the Dynamic Property's key")
 @DynamicRelationship(name = "Name from Dynamic Property", description = "FlowFiles that match the Dynamic Property's Regular Expression")
 public class RouteOnContent extends AbstractProcessor {
 
@@ -139,7 +141,7 @@ public class RouteOnContent extends AbstractProcessor {
                 .name(propertyDescriptorName)
                 .addValidator(StandardValidators.createRegexValidator(0, Integer.MAX_VALUE, true))
                 .dynamic(true)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
                 .build();
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteText.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteText.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteText.java
index edec3c1..5c3ed73 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteText.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteText.java
@@ -59,6 +59,7 @@ import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -81,7 +82,7 @@ import org.apache.nifi.processors.standard.util.NLKBufferedReader;
     + "The mechanism by which the text is compared to these user-defined properties is defined by the 'Matching Strategy'. The data is then routed according to these rules, routing "
     + "each line of the text individually.")
 @DynamicProperty(name = "Relationship Name", value = "value to match against", description = "Routes data that matches the value specified in the Dynamic Property Value to the "
-    + "Relationship specified in the Dynamic Property Key.")
+    + "Relationship specified in the Dynamic Property Key.", expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
 @DynamicRelationship(name = "Name from Dynamic Property", description = "FlowFiles that match the Dynamic Property's value")
 @WritesAttributes({
     @WritesAttribute(attribute = "RouteText.Route", description = "The name of the relationship to which the FlowFile was routed."),
@@ -160,7 +161,7 @@ public class RouteText extends AbstractProcessor {
         .name("Ignore Case")
         .description("If true, capitalization will not be taken into account when comparing values. E.g., matching against 'HELLO' or 'hello' will have the same result. "
             + "This property is ignored if the 'Matching Strategy' is set to 'Satisfies Expression'.")
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues("true", "false")
         .defaultValue("false")
         .required(true)
@@ -174,7 +175,7 @@ public class RouteText extends AbstractProcessor {
             + "(or neither line matches the Regular Expression). For example, to group together all lines in a CSV File by the first column, we can set this value to \"(.*?),.*\". "
             + "Two lines that have the same Group but different Relationships will never be placed into the same FlowFile.")
         .addValidator(StandardValidators.createRegexValidator(1, Integer.MAX_VALUE, false))
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(false)
         .build();
 
@@ -263,7 +264,7 @@ public class RouteText extends AbstractProcessor {
         return new PropertyDescriptor.Builder()
             .required(false)
             .name(propertyDescriptorName)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .dynamic(true)
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitRecord.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitRecord.java
index 947d997..2a5679d 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitRecord.java
@@ -37,6 +37,7 @@ import org.apache.nifi.annotation.behavior.WritesAttributes;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -86,7 +87,7 @@ public class SplitRecord extends AbstractProcessor {
         .name("Records Per Split")
         .description("Specifies how many records should be written to each 'split' or 'segment' FlowFile")
         .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .required(true)
         .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
index 13226fc..bc9c476 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
@@ -37,6 +37,7 @@ import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -134,7 +135,7 @@ public class TailFile extends AbstractProcessor {
             .name("tail-base-directory")
             .displayName("Base directory")
             .description("Base directory used to look for files to tail. This property is required when using Multifile mode.")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
             .required(false)
             .build();
@@ -144,7 +145,7 @@ public class TailFile extends AbstractProcessor {
             .displayName("Tailing mode")
             .description("Mode to use: single file will tail only one file, multiple file will look for a list of file. In Multiple mode"
                     + " the Base directory is required.")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(true)
             .allowableValues(MODE_SINGLEFILE, MODE_MULTIFILE)
             .defaultValue(MODE_SINGLEFILE.getValue())
@@ -156,7 +157,7 @@ public class TailFile extends AbstractProcessor {
             .description("Path of the file to tail in case of single file mode. If using multifile mode, regular expression to find files "
                     + "to tail in the base directory. In case recursivity is set to true, the regular expression will be used to match the "
                     + "path starting from the base directory (see additional details for examples).")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.createRegexValidator(0, Integer.MAX_VALUE, true))
             .required(true)
             .build();
@@ -169,7 +170,7 @@ public class TailFile extends AbstractProcessor {
                     + "(without extension), and will assume that the files that have rolled over live in the same directory as the file being tailed. "
                     + "The same glob pattern will be used for all files.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(false)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java
index 2e08012..b0a25f0 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java
@@ -51,6 +51,7 @@ import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
 import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -77,7 +78,8 @@ import com.google.common.cache.LoadingCache;
 @CapabilityDescription("Applies the provided XSLT file to the flowfile XML payload. A new FlowFile is created "
         + "with transformed content and is routed to the 'success' relationship. If the XSL transform "
         + "fails, the original FlowFile is routed to the 'failure' relationship")
-@DynamicProperty(name = "An XSLT transform parameter name", value = "An XSLT transform parameter value", supportsExpressionLanguage = true,
+@DynamicProperty(name = "An XSLT transform parameter name", value = "An XSLT transform parameter value",
+        expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
         description = "These XSLT parameters are passed to the transformer")
 public class TransformXml extends AbstractProcessor {
 
@@ -85,7 +87,7 @@ public class TransformXml extends AbstractProcessor {
             .name("XSLT file name")
             .description("Provides the name (including full path) of the XSLT file to apply to the flowfile XML content.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
             .build();
 
@@ -171,7 +173,7 @@ public class TransformXml extends AbstractProcessor {
     protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
         return new PropertyDescriptor.Builder()
                 .name(propertyDescriptorName)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
                 .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
                 .required(false)
                 .dynamic(true)


[13/15] nifi git commit: NIFI-4149 - Indicate if EL is evaluated against FFs or not - take into account input requirement for documentation rendering - Renamed variable registry scope and added comments - Doc + change in mock framework to check scope + u

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/AbstractKinesisStreamProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/AbstractKinesisStreamProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/AbstractKinesisStreamProcessor.java
index 79c8ded..e338226 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/AbstractKinesisStreamProcessor.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/AbstractKinesisStreamProcessor.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.processors.aws.kinesis.stream;
 
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processors.aws.kinesis.AbstractBaseKinesisProcessor;
@@ -35,7 +36,7 @@ public abstract class AbstractKinesisStreamProcessor extends AbstractBaseKinesis
             .name("kinesis-stream-name")
             .displayName("Amazon Kinesis Stream Name")
             .description("The name of Kinesis Stream")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/lambda/AbstractAWSLambdaProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/lambda/AbstractAWSLambdaProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/lambda/AbstractAWSLambdaProcessor.java
index 758191d..9b0050e 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/lambda/AbstractAWSLambdaProcessor.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/lambda/AbstractAWSLambdaProcessor.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.processors.aws.lambda;
 
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
@@ -34,7 +35,7 @@ public abstract class AbstractAWSLambdaProcessor extends AbstractAWSCredentialsP
     public static final PropertyDescriptor AWS_LAMBDA_FUNCTION_NAME = new PropertyDescriptor.Builder()
             .name("Amazon Lambda Name")
             .description("The Lambda Function Name")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
@@ -43,7 +44,7 @@ public abstract class AbstractAWSLambdaProcessor extends AbstractAWSCredentialsP
             .name("Amazon Lambda Qualifier (version)")
             .description("The Lambda Function Version")
             .defaultValue("$LATEST")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/s3/AbstractS3Processor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/s3/AbstractS3Processor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/s3/AbstractS3Processor.java
index d372532..5cc24e9 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/s3/AbstractS3Processor.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/s3/AbstractS3Processor.java
@@ -23,6 +23,7 @@ import java.util.List;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.util.StandardValidators;
@@ -47,7 +48,7 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
     public static final PropertyDescriptor FULL_CONTROL_USER_LIST = new PropertyDescriptor.Builder()
             .name("FullControl User List")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .description("A comma-separated list of Amazon User ID's or E-mail addresses that specifies who should have Full Control for an object")
             .defaultValue("${s3.permissions.full.users}")
@@ -55,7 +56,7 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
     public static final PropertyDescriptor READ_USER_LIST = new PropertyDescriptor.Builder()
             .name("Read Permission User List")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .description("A comma-separated list of Amazon User ID's or E-mail addresses that specifies who should have Read Access for an object")
             .defaultValue("${s3.permissions.read.users}")
@@ -63,7 +64,7 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
     public static final PropertyDescriptor WRITE_USER_LIST = new PropertyDescriptor.Builder()
             .name("Write Permission User List")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .description("A comma-separated list of Amazon User ID's or E-mail addresses that specifies who should have Write Access for an object")
             .defaultValue("${s3.permissions.write.users}")
@@ -71,7 +72,7 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
     public static final PropertyDescriptor READ_ACL_LIST = new PropertyDescriptor.Builder()
             .name("Read ACL User List")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .description("A comma-separated list of Amazon User ID's or E-mail addresses that specifies who should have permissions to read the Access Control List for an object")
             .defaultValue("${s3.permissions.readacl.users}")
@@ -79,7 +80,7 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
     public static final PropertyDescriptor WRITE_ACL_LIST = new PropertyDescriptor.Builder()
             .name("Write ACL User List")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .description("A comma-separated list of Amazon User ID's or E-mail addresses that specifies who should have permissions to change the Access Control List for an object")
             .defaultValue("${s3.permissions.writeacl.users}")
@@ -88,7 +89,7 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
             .name("canned-acl")
             .displayName("Canned ACL")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .description("Amazon Canned ACL for an object, one of: BucketOwnerFullControl, BucketOwnerRead, LogDeliveryWrite, AuthenticatedRead, PublicReadWrite, PublicRead, Private; " +
                     "will be ignored if any other ACL/permission/owner property is specified")
@@ -97,14 +98,14 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
     public static final PropertyDescriptor OWNER = new PropertyDescriptor.Builder()
             .name("Owner")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .description("The Amazon ID to use for the object's owner")
             .defaultValue("${s3.owner}")
             .build();
     public static final PropertyDescriptor BUCKET = new PropertyDescriptor.Builder()
             .name("Bucket")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
@@ -112,7 +113,7 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
             .name("Object Key")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("${filename}")
             .build();
     public static final PropertyDescriptor SIGNER_OVERRIDE = new PropertyDescriptor.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/sns/AbstractSNSProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/sns/AbstractSNSProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/sns/AbstractSNSProcessor.java
index cc0aa34..ba955e4 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/sns/AbstractSNSProcessor.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/sns/AbstractSNSProcessor.java
@@ -18,6 +18,7 @@ package org.apache.nifi.processors.aws.sns;
 
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
@@ -37,7 +38,7 @@ public abstract class AbstractSNSProcessor extends AbstractAWSCredentialsProvide
     public static final PropertyDescriptor ARN = new PropertyDescriptor.Builder()
             .name("Amazon Resource Name (ARN)")
             .description("The name of the resource to which notifications should be published")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
@@ -45,7 +46,7 @@ public abstract class AbstractSNSProcessor extends AbstractAWSCredentialsProvide
     public static final PropertyDescriptor ARN_TYPE = new PropertyDescriptor.Builder()
             .name("ARN Type")
             .description("The type of Amazon Resource Name that is being used.")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(true)
             .allowableValues(ARN_TYPE_TOPIC, ARN_TYPE_TARGET)
             .defaultValue(ARN_TYPE_TOPIC.getValue())

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/sqs/AbstractSQSProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/sqs/AbstractSQSProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/sqs/AbstractSQSProcessor.java
index dcae039..2d9841a 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/sqs/AbstractSQSProcessor.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/sqs/AbstractSQSProcessor.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.processors.aws.sqs;
 
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
@@ -40,7 +41,7 @@ public abstract class AbstractSQSProcessor extends AbstractAWSCredentialsProvide
             .name("Queue URL")
             .description("The URL of the queue to act upon")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(true)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/cloudwatch/PutCloudWatchMetric.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/cloudwatch/PutCloudWatchMetric.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/cloudwatch/PutCloudWatchMetric.java
index 533ca31..be84330 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/cloudwatch/PutCloudWatchMetric.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/cloudwatch/PutCloudWatchMetric.java
@@ -37,6 +37,7 @@ import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
 import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -62,7 +63,7 @@ import com.amazonaws.services.cloudwatch.model.StatisticSet;
         "minimum, maximum, sum and sample count.")
 @DynamicProperty(name = "Dimension Name", value = "Dimension Value",
         description = "Allows dimension name/value pairs to be added to the metric. AWS supports a maximum of 10 dimensions.",
-        supportsExpressionLanguage = true)
+        expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
 @Tags({"amazon", "aws", "cloudwatch", "metrics", "put", "publish"})
 public class PutCloudWatchMetric extends AbstractAWSCredentialsProviderProcessor<AmazonCloudWatchClient> {
 
@@ -93,7 +94,7 @@ public class PutCloudWatchMetric extends AbstractAWSCredentialsProviderProcessor
             .displayName("Namespace")
             .description("The namespace for the metric data for CloudWatch")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -101,7 +102,7 @@ public class PutCloudWatchMetric extends AbstractAWSCredentialsProviderProcessor
             .name("MetricName")
             .displayName("Metric Name")
             .description("The name of the metric")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(true)
             .addValidator(new StandardValidators.StringLengthValidator(1, 255))
             .build();
@@ -110,7 +111,7 @@ public class PutCloudWatchMetric extends AbstractAWSCredentialsProviderProcessor
             .name("Value")
             .displayName("Value")
             .description("The value for the metric. Must be a double")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(false)
             .addValidator(DOUBLE_VALIDATOR)
             .build();
@@ -119,7 +120,7 @@ public class PutCloudWatchMetric extends AbstractAWSCredentialsProviderProcessor
             .name("Timestamp")
             .displayName("Timestamp")
             .description("A point in time expressed as the number of milliseconds since Jan 1, 1970 00:00:00 UTC. If not specified, the default value is set to the time the metric data was received")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(false)
             .addValidator(StandardValidators.LONG_VALIDATOR)
             .build();
@@ -128,7 +129,7 @@ public class PutCloudWatchMetric extends AbstractAWSCredentialsProviderProcessor
             .name("Unit")
             .displayName("Unit")
             .description("The unit of the metric. (e.g Seconds, Bytes, Megabytes, Percent, Count,  Kilobytes/Second, Terabits/Second, Count/Second) For details see http://docs.aws.amazon.com/AmazonCloudWatch/latest/APIReference/API_MetricDatum.html")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
@@ -137,7 +138,7 @@ public class PutCloudWatchMetric extends AbstractAWSCredentialsProviderProcessor
             .name("maximum")
             .displayName("Maximum")
             .description("The maximum value of the sample set. Must be a double")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(false)
             .addValidator(DOUBLE_VALIDATOR)
             .build();
@@ -146,7 +147,7 @@ public class PutCloudWatchMetric extends AbstractAWSCredentialsProviderProcessor
             .name("minimum")
             .displayName("Minimum")
             .description("The minimum value of the sample set. Must be a double")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(false)
             .addValidator(DOUBLE_VALIDATOR)
             .build();
@@ -155,7 +156,7 @@ public class PutCloudWatchMetric extends AbstractAWSCredentialsProviderProcessor
             .name("sampleCount")
             .displayName("Sample Count")
             .description("The number of samples used for the statistic set. Must be a double")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(false)
             .addValidator(DOUBLE_VALIDATOR)
             .build();
@@ -164,7 +165,7 @@ public class PutCloudWatchMetric extends AbstractAWSCredentialsProviderProcessor
             .name("sum")
             .displayName("Sum")
             .description("The sum of values for the sample set. Must be a double")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(false)
             .addValidator(DOUBLE_VALIDATOR)
             .build();
@@ -188,7 +189,7 @@ public class PutCloudWatchMetric extends AbstractAWSCredentialsProviderProcessor
         return new PropertyDescriptor.Builder()
                 .name(propertyDescriptorName)
                 .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
                 .dynamic(true)
                 .build();
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/PutKinesisStream.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/PutKinesisStream.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/PutKinesisStream.java
index c6d79e0..14bdf82 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/PutKinesisStream.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/PutKinesisStream.java
@@ -35,6 +35,7 @@ import org.apache.nifi.annotation.behavior.WritesAttributes;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.ProcessContext;
@@ -72,7 +73,7 @@ public class PutKinesisStream extends AbstractKinesisStreamProcessor {
         .displayName("Amazon Kinesis Stream Partition Key")
         .name("amazon-kinesis-stream-partition-key")
         .description("The partition key attribute.  If it is not set, a random value is used")
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .defaultValue("${kinesis.partition.key}")
         .required(false)
         .addValidator(StandardValidators.ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR).build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
index 5cb207d..f2de794 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
@@ -33,6 +33,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -51,7 +52,7 @@ public class DeleteS3Object extends AbstractS3Processor {
             .name("Version")
             .description("The Version of the Object to delete")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(false)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/FetchS3Object.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/FetchS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/FetchS3Object.java
index 4be487e..61a3d06 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/FetchS3Object.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/FetchS3Object.java
@@ -33,6 +33,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.ProcessContext;
@@ -69,7 +70,7 @@ public class FetchS3Object extends AbstractS3Processor {
             .name("Version")
             .description("The Version of the Object to download")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(false)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/ListS3.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/ListS3.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/ListS3.java
index f2a094d..d988d16 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/ListS3.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/ListS3.java
@@ -40,6 +40,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.ProcessContext;
@@ -84,7 +85,7 @@ public class ListS3 extends AbstractS3Processor {
     public static final PropertyDescriptor DELIMITER = new PropertyDescriptor.Builder()
             .name("delimiter")
             .displayName("Delimiter")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .description("The string used to delimit directories within the bucket. Please consult the AWS documentation " +
@@ -94,7 +95,7 @@ public class ListS3 extends AbstractS3Processor {
     public static final PropertyDescriptor PREFIX = new PropertyDescriptor.Builder()
             .name("prefix")
             .displayName("Prefix")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .description("The prefix used to filter the object list. In most cases, it should end with a forward slash ('/').")
@@ -103,7 +104,7 @@ public class ListS3 extends AbstractS3Processor {
     public static final PropertyDescriptor USE_VERSIONS = new PropertyDescriptor.Builder()
             .name("use-versions")
             .displayName("Use Versions")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(true)
             .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
             .allowableValues("true", "false")
@@ -114,7 +115,7 @@ public class ListS3 extends AbstractS3Processor {
     public static final PropertyDescriptor LIST_TYPE = new PropertyDescriptor.Builder()
             .name("list-type")
             .displayName("List Type")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(true)
             .addValidator(StandardValidators.INTEGER_VALIDATOR)
             .allowableValues(

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/PutS3Object.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/PutS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/PutS3Object.java
index 817b2e5..3dc3199 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/PutS3Object.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/PutS3Object.java
@@ -50,6 +50,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.DataUnit;
@@ -104,7 +105,7 @@ import com.amazonaws.services.s3.model.UploadPartResult;
 @DynamicProperty(name = "The name of a User-Defined Metadata field to add to the S3 Object",
         value = "The value of a User-Defined Metadata field to add to the S3 Object",
         description = "Allows user-defined metadata to be added to the S3 object as key/value pairs",
-        supportsExpressionLanguage = true)
+        expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
 @ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the S3 object")
 @WritesAttributes({
     @WritesAttribute(attribute = "s3.bucket", description = "The S3 bucket where the Object was put in S3"),
@@ -129,7 +130,7 @@ public class PutS3Object extends AbstractS3Processor {
     public static final PropertyDescriptor EXPIRATION_RULE_ID = new PropertyDescriptor.Builder()
         .name("Expiration Time Rule")
         .required(false)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .build();
 
@@ -143,7 +144,7 @@ public class PutS3Object extends AbstractS3Processor {
                 "no content type is provided and cannot be determined by the filename, the default content type " +
                 "\"application/octet-stream\" will be used.")
         .required(false)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .build();
 
@@ -237,7 +238,7 @@ public class PutS3Object extends AbstractS3Processor {
         return new PropertyDescriptor.Builder()
             .name(propertyDescriptorName)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .dynamic(true)
             .build();
     }
@@ -408,7 +409,7 @@ public class PutS3Object extends AbstractS3Processor {
         /*
          * If necessary, run age off for existing uploads in AWS S3 and local state
          */
-        ageoffS3Uploads(context, s3, now);
+        ageoffS3Uploads(context, s3, now, bucket);
 
         /*
          * Then
@@ -717,16 +718,15 @@ public class PutS3Object extends AbstractS3Processor {
     private final AtomicLong lastS3AgeOff = new AtomicLong(0L);
     private final DateFormat logFormat = new SimpleDateFormat();
 
-    protected void ageoffS3Uploads(final ProcessContext context, final AmazonS3Client s3, final long now) {
-        MultipartUploadListing oldUploads = getS3AgeoffListAndAgeoffLocalState(context, s3, now);
+    protected void ageoffS3Uploads(final ProcessContext context, final AmazonS3Client s3, final long now, String bucket) {
+        MultipartUploadListing oldUploads = getS3AgeoffListAndAgeoffLocalState(context, s3, now, bucket);
         for (MultipartUpload upload : oldUploads.getMultipartUploads()) {
             abortS3MultipartUpload(s3, oldUploads.getBucketName(), upload);
         }
     }
 
-    protected MultipartUploadListing getS3AgeoffListAndAgeoffLocalState(final ProcessContext context, final AmazonS3Client s3, final long now) {
+    protected MultipartUploadListing getS3AgeoffListAndAgeoffLocalState(final ProcessContext context, final AmazonS3Client s3, final long now, String bucket) {
         final long ageoff_interval = context.getProperty(MULTIPART_S3_AGEOFF_INTERVAL).asTimePeriod(TimeUnit.MILLISECONDS);
-        final String bucket = context.getProperty(BUCKET).evaluateAttributeExpressions().getValue();
         final Long maxAge = context.getProperty(MULTIPART_S3_MAX_AGE).asTimePeriod(TimeUnit.MILLISECONDS);
         final long ageCutoff = now - maxAge;
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sns/PutSNS.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sns/PutSNS.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sns/PutSNS.java
index 0ca2be1..4a93243 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sns/PutSNS.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sns/PutSNS.java
@@ -31,6 +31,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -53,7 +54,7 @@ public class PutSNS extends AbstractSNSProcessor {
             .name("Character Set")
             .description("The character set in which the FlowFile's content is encoded")
             .defaultValue("UTF-8")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
             .required(true)
             .build();
@@ -69,7 +70,7 @@ public class PutSNS extends AbstractSNSProcessor {
     public static final PropertyDescriptor SUBJECT = new PropertyDescriptor.Builder()
             .name("E-mail Subject")
             .description("The optional subject to use for any subscribers that are subscribed via E-mail")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
@@ -90,7 +91,7 @@ public class PutSNS extends AbstractSNSProcessor {
         return new PropertyDescriptor.Builder()
                 .name(propertyDescriptorName)
                 .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
                 .required(false)
                 .dynamic(true)
                 .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/DeleteSQS.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/DeleteSQS.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/DeleteSQS.java
index 47be9e8..5beead5 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/DeleteSQS.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/DeleteSQS.java
@@ -28,6 +28,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -47,7 +48,7 @@ public class DeleteSQS extends AbstractSQSProcessor {
     public static final PropertyDescriptor RECEIPT_HANDLE = new PropertyDescriptor.Builder()
             .name("Receipt Handle")
             .description("The identifier that specifies the receipt of the message")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .defaultValue("${sqs.receipt.handle}")

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/GetSQS.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/GetSQS.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/GetSQS.java
index eebc731..17f8c4b 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/GetSQS.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/GetSQS.java
@@ -38,6 +38,7 @@ import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -85,7 +86,7 @@ public class GetSQS extends AbstractSQSProcessor {
     public static final PropertyDescriptor VISIBILITY_TIMEOUT = new PropertyDescriptor.Builder()
             .name("Visibility Timeout")
             .description("The amount of time after a message is received but not deleted that the message is hidden from other consumers")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(true)
             .defaultValue("15 mins")
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
@@ -101,7 +102,7 @@ public class GetSQS extends AbstractSQSProcessor {
 
     public static final PropertyDescriptor DYNAMIC_QUEUE_URL = new PropertyDescriptor.Builder()
             .fromPropertyDescriptor(QUEUE_URL)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(ResultType.STRING, true))
             .build();
 
@@ -109,7 +110,7 @@ public class GetSQS extends AbstractSQSProcessor {
             .name("Receive Message Wait Time")
             .description("The maximum amount of time to wait on a long polling receive call. Setting this to a value of 1 second or greater will "
                 + "reduce the number of SQS requests and decrease fetch latency at the cost of a constantly active thread.")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(true)
             .defaultValue("0 sec")
             .addValidator(StandardValidators.createTimePeriodValidator(0, TimeUnit.SECONDS, 20, TimeUnit.SECONDS))  // 20 seconds is the maximum allowed by SQS

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/PutSQS.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/PutSQS.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/PutSQS.java
index 29efc0b..a203da0 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/PutSQS.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/PutSQS.java
@@ -36,6 +36,7 @@ import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -53,7 +54,7 @@ import com.amazonaws.services.sqs.model.SendMessageBatchRequestEntry;
 @CapabilityDescription("Publishes a message to an Amazon Simple Queuing Service Queue")
 @DynamicProperty(name = "The name of a Message Attribute to add to the message", value = "The value of the Message Attribute",
         description = "Allows the user to add key/value pairs as Message Attributes by adding a property whose name will become the name of "
-        + "the Message Attribute and value will become the value of the Message Attribute", supportsExpressionLanguage = true)
+        + "the Message Attribute and value will become the value of the Message Attribute", expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
 public class PutSQS extends AbstractSQSProcessor {
 
     public static final PropertyDescriptor DELAY = new PropertyDescriptor.Builder()
@@ -78,7 +79,7 @@ public class PutSQS extends AbstractSQSProcessor {
     protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
         return new PropertyDescriptor.Builder()
                 .name(propertyDescriptorName)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
                 .required(false)
                 .dynamic(true)
                 .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/cloudwatch/ITPutCloudWatchMetric.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/cloudwatch/ITPutCloudWatchMetric.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/cloudwatch/ITPutCloudWatchMetric.java
index d2d2afe..31fcb87 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/cloudwatch/ITPutCloudWatchMetric.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/cloudwatch/ITPutCloudWatchMetric.java
@@ -16,6 +16,8 @@
  */
 package org.apache.nifi.processors.aws.cloudwatch;
 
+import java.io.IOException;
+
 import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
 import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
 import org.apache.nifi.processors.aws.sns.PutSNS;
@@ -23,8 +25,6 @@ import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.Test;
 
-import java.io.IOException;
-
 /**
  * Provides integration level testing with actual AWS CloudWatch resources for {@link PutCloudWatchMetric} and requires additional configuration and resources to work.
  */
@@ -50,7 +50,6 @@ public class ITPutCloudWatchMetric {
     @Test
     public void testPublishWithCredentialsProviderService() throws Throwable {
         final TestRunner runner = TestRunners.newTestRunner(new PutCloudWatchMetric());
-        runner.setValidateExpressionUsage(false);
 
         final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
         runner.addControllerService("awsCredentialsProvider", serviceImpl);

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITPutS3Object.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITPutS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITPutS3Object.java
index 2ac2d94..2832e29 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITPutS3Object.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITPutS3Object.java
@@ -822,13 +822,13 @@ public class ITPutS3Object extends AbstractS3IT {
         // initiation times in whole seconds.
         Long now = System.currentTimeMillis();
 
-        MultipartUploadListing uploadList = processor.getS3AgeoffListAndAgeoffLocalState(context, client, now);
+        MultipartUploadListing uploadList = processor.getS3AgeoffListAndAgeoffLocalState(context, client, now, BUCKET_NAME);
         Assert.assertEquals(3, uploadList.getMultipartUploads().size());
 
         MultipartUpload upload0 = uploadList.getMultipartUploads().get(0);
         processor.abortS3MultipartUpload(client, BUCKET_NAME, upload0);
 
-        uploadList = processor.getS3AgeoffListAndAgeoffLocalState(context, client, now+1000);
+        uploadList = processor.getS3AgeoffListAndAgeoffLocalState(context, client, now+1000, BUCKET_NAME);
         Assert.assertEquals(2, uploadList.getMultipartUploads().size());
 
         final Map<String, String> attrs = new HashMap<>();
@@ -836,7 +836,7 @@ public class ITPutS3Object extends AbstractS3IT {
         runner.enqueue(getResourcePath(SAMPLE_FILE_RESOURCE_NAME), attrs);
         runner.run();
 
-        uploadList = processor.getS3AgeoffListAndAgeoffLocalState(context, client, now+2000);
+        uploadList = processor.getS3AgeoffListAndAgeoffLocalState(context, client, now+2000, BUCKET_NAME);
         Assert.assertEquals(0, uploadList.getMultipartUploads().size());
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sns/ITPutSNS.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sns/ITPutSNS.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sns/ITPutSNS.java
index 94fb3d3..2c8fe79 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sns/ITPutSNS.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sns/ITPutSNS.java
@@ -16,18 +16,18 @@
  */
 package org.apache.nifi.processors.aws.sns;
 
-import org.apache.nifi.util.TestRunner;
-import org.apache.nifi.util.TestRunners;
-import org.junit.Test;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.nio.file.Paths;
 import java.util.HashMap;
 import java.util.Map;
 
-import static org.junit.Assert.assertTrue;
 import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
 import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Test;
 
 /**
  * Provides integration level testing with actual AWS S3 resources for {@link PutSNS} and requires additional configuration and resources to work.
@@ -39,7 +39,6 @@ public class ITPutSNS {
     @Test
     public void testPublish() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new PutSNS());
-        runner.setValidateExpressionUsage(false);
         runner.setProperty(PutSNS.CREDENTIALS_FILE, CREDENTIALS_FILE);
         runner.setProperty(PutSNS.ARN, "arn:aws:sns:us-west-2:100515378163:test-topic-1");
         assertTrue(runner.setProperty("DynamicProperty", "hello!").isValid());
@@ -56,7 +55,6 @@ public class ITPutSNS {
     public void testPublishWithCredentialsProviderService() throws Throwable {
         final TestRunner runner = TestRunners.newTestRunner(new PutSNS());
         String snsArn = "Add Sns arn here";
-        runner.setValidateExpressionUsage(false);
         runner.setProperty(PutSNS.ARN, snsArn);
         assertTrue(runner.setProperty("DynamicProperty", "hello!").isValid());
         final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sns/TestPutSNS.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sns/TestPutSNS.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sns/TestPutSNS.java
index 663af33..33eba23 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sns/TestPutSNS.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sns/TestPutSNS.java
@@ -16,6 +16,9 @@
  */
 package org.apache.nifi.processors.aws.sns;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.List;
@@ -25,19 +28,15 @@ import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
-
-import com.amazonaws.services.sns.AmazonSNSClient;
-import com.amazonaws.services.sns.model.AmazonSNSException;
-import com.amazonaws.services.sns.model.PublishRequest;
-import com.amazonaws.services.sns.model.PublishResult;
-
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 import org.mockito.Mockito;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import com.amazonaws.services.sns.AmazonSNSClient;
+import com.amazonaws.services.sns.model.AmazonSNSException;
+import com.amazonaws.services.sns.model.PublishRequest;
+import com.amazonaws.services.sns.model.PublishResult;
 
 
 public class TestPutSNS {
@@ -51,6 +50,7 @@ public class TestPutSNS {
     public void setUp() {
         mockSNSClient = Mockito.mock(AmazonSNSClient.class);
         mockPutSNS = new PutSNS() {
+            @Override
             protected AmazonSNSClient getClient() {
                 actualSNSClient = client;
                 return mockSNSClient;
@@ -61,7 +61,6 @@ public class TestPutSNS {
 
     @Test
     public void testPublish() throws IOException {
-        runner.setValidateExpressionUsage(false);
         runner.setProperty(PutSNS.CREDENTIALS_FILE, "src/test/resources/mock-aws-credentials.properties");
         runner.setProperty(PutSNS.ARN, "arn:aws:sns:us-west-2:123456789012:test-topic-1");
         runner.setProperty(PutSNS.SUBJECT, "${eval.subject}");
@@ -92,7 +91,6 @@ public class TestPutSNS {
 
     @Test
     public void testPublishFailure() throws IOException {
-        runner.setValidateExpressionUsage(false);
         runner.setProperty(PutSNS.ARN, "arn:aws:sns:us-west-2:123456789012:test-topic-1");
         final Map<String, String> ffAttributes = new HashMap<>();
         ffAttributes.put("filename", "1.txt");

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sqs/ITPutSQS.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sqs/ITPutSQS.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sqs/ITPutSQS.java
index 69544a2..ba9bfaa 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sqs/ITPutSQS.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sqs/ITPutSQS.java
@@ -60,7 +60,6 @@ public class ITPutSQS {
         runner.setProperty(PutSQS.TIMEOUT, "30 secs");
         String queueUrl = "Add queue url here";
         runner.setProperty(PutSQS.QUEUE_URL, queueUrl);
-        runner.setValidateExpressionUsage(false);
         final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
 
         runner.addControllerService("awsCredentialsProvider", serviceImpl);

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sqs/TestPutSQS.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sqs/TestPutSQS.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sqs/TestPutSQS.java
index 39bef8b..39dae75 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sqs/TestPutSQS.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sqs/TestPutSQS.java
@@ -16,25 +16,24 @@
  */
 package org.apache.nifi.processors.aws.sqs;
 
+import static org.junit.Assert.assertEquals;
+
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
-
-import com.amazonaws.services.sqs.AmazonSQSClient;
-import com.amazonaws.services.sqs.model.AmazonSQSException;
-import com.amazonaws.services.sqs.model.SendMessageBatchRequest;
-import com.amazonaws.services.sqs.model.SendMessageBatchResult;
-
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 import org.mockito.Mockito;
 
-import static org.junit.Assert.assertEquals;
+import com.amazonaws.services.sqs.AmazonSQSClient;
+import com.amazonaws.services.sqs.model.AmazonSQSException;
+import com.amazonaws.services.sqs.model.SendMessageBatchRequest;
+import com.amazonaws.services.sqs.model.SendMessageBatchResult;
 
 
 public class TestPutSQS {
@@ -48,6 +47,7 @@ public class TestPutSQS {
     public void setUp() {
         mockSQSClient = Mockito.mock(AmazonSQSClient.class);
         mockPutSQS = new PutSQS() {
+            @Override
             protected AmazonSQSClient getClient() {
                 actualSQSClient = client;
                 return mockSQSClient;
@@ -58,7 +58,6 @@ public class TestPutSQS {
 
     @Test
     public void testSimplePut() throws IOException {
-        runner.setValidateExpressionUsage(false);
         runner.setProperty(PutSQS.QUEUE_URL, "https://sqs.us-west-2.amazonaws.com/123456789012/test-queue-000000000");
         Assert.assertTrue(runner.setProperty("x-custom-prop", "hello").isValid());
 
@@ -83,7 +82,6 @@ public class TestPutSQS {
 
     @Test
     public void testPutException() throws IOException {
-        runner.setValidateExpressionUsage(false);
         runner.setProperty(PutSQS.QUEUE_URL, "https://sqs.us-west-2.amazonaws.com/123456789012/test-queue-000000000");
 
         final Map<String, String> attrs = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/AbstractAzureBlobProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/AbstractAzureBlobProcessor.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/AbstractAzureBlobProcessor.java
index cab0ed9..5b2d8ce 100644
--- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/AbstractAzureBlobProcessor.java
+++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/AbstractAzureBlobProcessor.java
@@ -19,6 +19,7 @@ package org.apache.nifi.processors.azure;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.util.StandardValidators;
@@ -33,10 +34,24 @@ import java.util.Set;
 
 public abstract class AbstractAzureBlobProcessor extends AbstractProcessor {
 
-    public static final PropertyDescriptor BLOB = new PropertyDescriptor.Builder().name("blob").displayName("Blob").description("The filename of the blob")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR).expressionLanguageSupported(true).required(true).defaultValue("${azure.blobname}").build();
-    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success").description("All successfully processed FlowFiles are routed to this relationship").build();
-    public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure").description("Unsuccessful operations will be transferred to the failure relationship.").build();
+    public static final PropertyDescriptor BLOB = new PropertyDescriptor.Builder()
+            .name("blob")
+            .displayName("Blob")
+            .description("The filename of the blob")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(true)
+            .defaultValue("${azure.blobname}")
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("All successfully processed FlowFiles are routed to this relationship")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Unsuccessful operations will be transferred to the failure relationship.")
+            .build();
 
     private static final List<PropertyDescriptor> PROPERTIES = Collections
             .unmodifiableList(Arrays.asList(

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/ConsumeAzureEventHub.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/ConsumeAzureEventHub.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/ConsumeAzureEventHub.java
index dc18097..4eb0da5 100644
--- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/ConsumeAzureEventHub.java
+++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/ConsumeAzureEventHub.java
@@ -37,6 +37,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.logging.ComponentLog;
@@ -95,7 +96,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
             .displayName("Event Hub Namespace")
             .description("The Azure Namespace that the Event Hub is assigned to. This is generally equal to <Event Hub Name>-ns.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .required(true)
             .build();
     static final PropertyDescriptor EVENT_HUB_NAME = new PropertyDescriptor.Builder()
@@ -103,7 +104,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
             .displayName("Event Hub Name")
             .description("The name of the Azure Event Hub to pull messages from.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .required(true)
             .build();
     // TODO: Do we need to support custom service endpoints as GetAzureEventHub does? Is it possible?
@@ -112,7 +113,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
             .displayName("Shared Access Policy Name")
             .description("The name of the Event Hub Shared Access Policy. This Policy must have Listen permissions.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .required(true)
             .build();
     static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
@@ -120,7 +121,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
             .displayName("Shared Access Policy Primary Key")
             .description("The primary key of the Event Hub Shared Access Policy.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .sensitive(true)
             .required(true)
             .build();
@@ -129,7 +130,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
             .displayName("Event Hub Consumer Group")
             .description("The name of the Event Hub Consumer Group to use.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue("$Default")
             .required(true)
             .build();
@@ -139,7 +140,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
             .description("The hostname of this Event Hub Consumer instance." +
                     " If not specified, an unique identifier is generated in 'nifi-<UUID>' format.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .required(false)
             .build();
 
@@ -149,7 +150,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
             .description("The Record Reader to use for reading received messages." +
                     " The Event Hub name can be referred by Expression Language '${eventhub.name}' to access a schema.")
             .identifiesControllerService(RecordReaderFactory.class)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(false)
             .build();
     static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
@@ -159,7 +160,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
                     " The Event Hub name can be referred by Expression Language '${eventhub.name}' to access a schema." +
                     " If not specified, each message will create a FlowFile.")
             .identifiesControllerService(RecordSetWriterFactory.class)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(false)
             .build();
 
@@ -188,7 +189,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
                     " https://github.com/Azure/azure-event-hubs-java/issues/125")
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
             .defaultValue("300")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .required(true)
             .build();
     static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
@@ -202,7 +203,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
                     " The higher number, the higher throughput, but possibly less consistent.")
             .defaultValue("10")
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .required(true)
             .build();
     static final PropertyDescriptor RECEIVE_TIMEOUT = new PropertyDescriptor.Builder()
@@ -211,7 +212,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
             .description("The amount of time this consumer should wait to receive the Prefetch Count before returning.")
             .defaultValue("1 min")
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .required(true)
             .build();
     static final PropertyDescriptor STORAGE_ACCOUNT_NAME = new PropertyDescriptor.Builder()
@@ -219,7 +220,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
             .displayName("Storage Account Name")
             .description("Name of the Azure Storage account to store Event Hub Consumer Group state.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .required(true)
             .build();
     static final PropertyDescriptor STORAGE_ACCOUNT_KEY = new PropertyDescriptor.Builder()
@@ -228,7 +229,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
             .description("The Azure Storage account key to store Event Hub Consumer Group state.")
             .sensitive(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .required(true)
             .build();
     static final PropertyDescriptor STORAGE_CONTAINER_NAME = new PropertyDescriptor.Builder()
@@ -237,7 +238,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
             .description("Name of the Azure Storage Container to store Event Hub Consumer Group state." +
                     " If not specified, Event Hub name is used.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .required(false)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java
index c40707e..3334703 100644
--- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java
+++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java
@@ -30,6 +30,7 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -80,14 +81,14 @@ public class GetAzureEventHub extends AbstractProcessor {
             .name("Event Hub Namespace")
             .description("The Azure Namespace that the Event Hub is assigned to. This is generally equal to <Event Hub Name>-ns")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(true)
             .build();
     static final PropertyDescriptor SERVICE_BUS_ENDPOINT = new PropertyDescriptor.Builder()
             .name("Service Bus Endpoint")
             .description("To support Namespaces in non-standard Host URIs ( not .servicebus.windows.net,  ie .servicebus.chinacloudapi.cn) select from the drop down acceptable options ")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .allowableValues(".servicebus.windows.net",".servicebus.chinacloudapi.cn")
             .defaultValue(".servicebus.windows.net")
             .required(true)
@@ -96,14 +97,14 @@ public class GetAzureEventHub extends AbstractProcessor {
             .name("Shared Access Policy Name")
             .description("The name of the Event Hub Shared Access Policy. This Policy must have Listen permissions.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(true)
             .build();
     static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
             .name("Shared Access Policy Primary Key")
             .description("The primary key of the Event Hub Shared Access Policy")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .sensitive(true)
             .required(true)
             .build();
@@ -113,14 +114,14 @@ public class GetAzureEventHub extends AbstractProcessor {
             .description("The number of partitions that the Event Hub has. Only this number of partitions will be used, "
                     + "so it is important to ensure that if the number of partitions changes that this value be updated. Otherwise, some messages may not be consumed.")
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(true)
             .build();
     static final PropertyDescriptor CONSUMER_GROUP = new PropertyDescriptor.Builder()
             .name("Event Hub Consumer Group")
             .description("The name of the Event Hub Consumer Group to use when pulling events")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .defaultValue("$Default")
             .required(true)
             .build();
@@ -130,21 +131,21 @@ public class GetAzureEventHub extends AbstractProcessor {
             .description("A timestamp (ISO-8061 Instant) formatted as YYYY-MM-DDThhmmss.sssZ (2016-01-01T01:01:01.000Z) from which messages "
                     + "should have been enqueued in the EventHub to start reading from")
             .addValidator(StandardValidators.ISO8061_INSTANT_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(false)
             .build();
     static final PropertyDescriptor RECEIVER_FETCH_SIZE = new PropertyDescriptor.Builder()
             .name("Partition Recivier Fetch Size")
             .description("The number of events that a receiver should fetch from an EventHubs partition before returning. Default(100)")
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(false)
             .build();
     static final PropertyDescriptor RECEIVER_FETCH_TIMEOUT = new PropertyDescriptor.Builder()
             .name("Partiton Receiver Timeout (millseconds)")
             .description("The amount of time a Partition Receiver should wait to receive the Fetch Size before returning. Default(60000)")
             .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(false)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/PutAzureEventHub.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/PutAzureEventHub.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/PutAzureEventHub.java
index 703bfdb..6ebed60 100644
--- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/PutAzureEventHub.java
+++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/PutAzureEventHub.java
@@ -31,6 +31,7 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -70,21 +71,21 @@ public class PutAzureEventHub extends AbstractProcessor {
             .name("Event Hub Namespace")
             .description("The Azure Namespace that the Event Hub is assigned to. This is generally equal to <Event Hub Name>-ns")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(true)
             .build();
     static final PropertyDescriptor ACCESS_POLICY = new PropertyDescriptor.Builder()
             .name("Shared Access Policy Name")
             .description("The name of the Event Hub Shared Access Policy. This Policy must have Send permissions.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(true)
             .build();
     static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
             .name("Shared Access Policy Primary Key")
             .description("The primary key of the Event Hub Shared Access Policy")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .sensitive(true)
             .required(true)
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureBlobStorage.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureBlobStorage.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureBlobStorage.java
index 1cba6dd..fd27958 100644
--- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureBlobStorage.java
+++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureBlobStorage.java
@@ -38,6 +38,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processor.util.list.AbstractListProcessor;
@@ -78,9 +79,14 @@ import java.util.Map;
         "where the previous node left off, without duplicating the data.")
 public class ListAzureBlobStorage extends AbstractListProcessor<BlobInfo> {
 
-    private static final PropertyDescriptor PROP_PREFIX = new PropertyDescriptor.Builder().name("prefix").displayName("Prefix").description("Search prefix for listing")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR).expressionLanguageSupported(true).required(false).build();
-
+    private static final PropertyDescriptor PROP_PREFIX = new PropertyDescriptor.Builder()
+            .name("prefix")
+            .displayName("Prefix")
+            .description("Search prefix for listing")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false)
+            .build();
 
     private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
             AzureStorageUtils.CONTAINER,

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureStorageUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureStorageUtils.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureStorageUtils.java
index 95b6d7e..0a9696e 100644
--- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureStorageUtils.java
+++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureStorageUtils.java
@@ -24,6 +24,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.ProcessContext;
@@ -40,7 +41,9 @@ public final class AzureStorageUtils {
     public static final String BLOCK = "Block";
     public static final String PAGE = "Page";
 
-    public static final PropertyDescriptor ACCOUNT_KEY = new PropertyDescriptor.Builder().name("storage-account-key").displayName("Storage Account Key")
+    public static final PropertyDescriptor ACCOUNT_KEY = new PropertyDescriptor.Builder()
+            .name("storage-account-key")
+            .displayName("Storage Account Key")
             .description("The storage account key. This is an admin-like password providing access to every container in this account. It is recommended " +
                     "one uses Shared Access Signature (SAS) token instead for fine-grained control with policies. " +
                     "There are certain risks in allowing the account key to be stored as a flowfile " +
@@ -48,18 +51,34 @@ public final class AzureStorageUtils {
                     "be fetched dynamically from a flow file attribute, care must be taken to restrict access to " +
                     "the event provenance data (e.g. by strictly controlling the policies governing provenance for this Processor). " +
                     "In addition, the provenance repositories may be put on encrypted disk partitions.")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR).expressionLanguageSupported(true).required(false).sensitive(true).build();
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .sensitive(true)
+            .build();
 
-    public static final PropertyDescriptor ACCOUNT_NAME = new PropertyDescriptor.Builder().name("storage-account-name").displayName("Storage Account Name")
+    public static final PropertyDescriptor ACCOUNT_NAME = new PropertyDescriptor.Builder()
+            .name("storage-account-name")
+            .displayName("Storage Account Name")
             .description("The storage account name.  There are certain risks in allowing the account name to be stored as a flowfile " +
                     "attribute. While it does provide for a more flexible flow by allowing the account name to " +
                     "be fetched dynamically from a flowfile attribute, care must be taken to restrict access to " +
                     "the event provenance data (e.g. by strictly controlling the policies governing provenance for this Processor). " +
                     "In addition, the provenance repositories may be put on encrypted disk partitions.")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR).expressionLanguageSupported(true).required(true).sensitive(true).build();
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(true)
+            .sensitive(true)
+            .build();
 
-    public static final PropertyDescriptor CONTAINER = new PropertyDescriptor.Builder().name("container-name").displayName("Container Name")
-            .description("Name of the Azure storage container").addValidator(StandardValidators.NON_EMPTY_VALIDATOR).expressionLanguageSupported(true).required(true).build();
+    public static final PropertyDescriptor CONTAINER = new PropertyDescriptor.Builder()
+            .name("container-name")
+            .displayName("Container Name")
+            .description("Name of the Azure storage container")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(true)
+            .build();
 
     public static final PropertyDescriptor PROP_SAS_TOKEN = new PropertyDescriptor.Builder()
             .name("storage-sas-token")
@@ -71,7 +90,7 @@ public final class AzureStorageUtils {
                     "the event provenance data (e.g. by strictly controlling the policies governing provenance for this Processor). " +
                     "In addition, the provenance repositories may be put on encrypted disk partitions.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .sensitive(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();


[06/15] nifi git commit: NIFI-4149 - Indicate if EL is evaluated against FFs or not - take into account input requirement for documentation rendering - Renamed variable registry scope and added comments - Doc + change in mock framework to check scope + u

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java b/nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java
index 659dd2f..dbfbcbb 100644
--- a/nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java
+++ b/nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java
@@ -24,6 +24,7 @@ import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.annotation.behavior.WritesAttribute;
 import org.apache.nifi.annotation.behavior.WritesAttributes;
@@ -94,7 +95,7 @@ public class ConsumeMQTT extends AbstractMQTTProcessor {
             .name("Topic Filter")
             .description("The MQTT topic filter to designate the topics to subscribe to.")
             .required(true)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java b/nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java
index debe00a..c1f1a68 100644
--- a/nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java
+++ b/nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java
@@ -23,6 +23,7 @@ import org.apache.nifi.annotation.behavior.SystemResource;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
@@ -66,7 +67,7 @@ public class PublishMQTT extends AbstractMQTTProcessor {
     public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
             .name("Topic")
             .description("The topic to publish the message to.")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(true)
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
@@ -77,7 +78,7 @@ public class PublishMQTT extends AbstractMQTTProcessor {
             .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'. " +
                     "Expression language is allowed in order to support publishing messages with different QoS but the end value of the property must be either '0', '1' or '2'. ")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(QOS_VALIDATOR)
             .build();
 
@@ -85,7 +86,7 @@ public class PublishMQTT extends AbstractMQTTProcessor {
             .name("Retain Message")
             .description("Whether or not the retain flag should be set on the MQTT message.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(RETAIN_VALIDATOR)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/PutParquet.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/PutParquet.java b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/PutParquet.java
index e5d2446..8b8b814 100644
--- a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/PutParquet.java
+++ b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/PutParquet.java
@@ -32,6 +32,7 @@ import org.apache.nifi.avro.AvroTypeUtil;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.ProcessContext;
@@ -82,7 +83,7 @@ public class PutParquet extends AbstractPutHDFSRecord {
             .description("The row group size used by the Parquet writer. " +
                     "The value is specified in the format of <Data Size> <Data Unit> where Data Unit is one of B, KB, MB, GB, TB.")
             .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
@@ -91,7 +92,7 @@ public class PutParquet extends AbstractPutHDFSRecord {
             .description("The page size used by the Parquet writer. " +
                     "The value is specified in the format of <Data Size> <Data Unit> where Data Unit is one of B, KB, MB, GB, TB.")
             .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor DICTIONARY_PAGE_SIZE = new PropertyDescriptor.Builder()
@@ -100,7 +101,7 @@ public class PutParquet extends AbstractPutHDFSRecord {
             .description("The dictionary page size used by the Parquet writer. " +
                     "The value is specified in the format of <Data Size> <Data Unit> where Data Unit is one of B, KB, MB, GB, TB.")
             .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor MAX_PADDING_SIZE = new PropertyDescriptor.Builder()
@@ -110,7 +111,7 @@ public class PutParquet extends AbstractPutHDFSRecord {
                     "underlying filesystem. If the underlying filesystem is not a block filesystem like HDFS, this has no effect. " +
                     "The value is specified in the format of <Data Size> <Data Unit> where Data Unit is one of B, KB, MB, GB, TB.")
             .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor ENABLE_DICTIONARY_ENCODING = new PropertyDescriptor.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-poi-bundle/nifi-poi-processors/src/main/java/org/apache/nifi/processors/poi/ConvertExcelToCSVProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-poi-bundle/nifi-poi-processors/src/main/java/org/apache/nifi/processors/poi/ConvertExcelToCSVProcessor.java b/nifi-nar-bundles/nifi-poi-bundle/nifi-poi-processors/src/main/java/org/apache/nifi/processors/poi/ConvertExcelToCSVProcessor.java
index 03e79c3..d88b256 100644
--- a/nifi-nar-bundles/nifi-poi-bundle/nifi-poi-processors/src/main/java/org/apache/nifi/processors/poi/ConvertExcelToCSVProcessor.java
+++ b/nifi-nar-bundles/nifi-poi-bundle/nifi-poi-processors/src/main/java/org/apache/nifi/processors/poi/ConvertExcelToCSVProcessor.java
@@ -37,6 +37,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.csv.CSVUtils;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -96,7 +97,7 @@ public class ConvertExcelToCSVProcessor
                     " is left blank then all of the sheets will be extracted from the Excel document. The list of names is case in-sensitive. Any sheets not " +
                     "specified in this value will be ignored.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-redis-bundle/nifi-redis-extensions/src/main/java/org/apache/nifi/redis/util/RedisUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-redis-bundle/nifi-redis-extensions/src/main/java/org/apache/nifi/redis/util/RedisUtils.java b/nifi-nar-bundles/nifi-redis-bundle/nifi-redis-extensions/src/main/java/org/apache/nifi/redis/util/RedisUtils.java
index 229c438..aed823b 100644
--- a/nifi-nar-bundles/nifi-redis-bundle/nifi-redis-extensions/src/main/java/org/apache/nifi/redis/util/RedisUtils.java
+++ b/nifi-nar-bundles/nifi-redis-bundle/nifi-redis-extensions/src/main/java/org/apache/nifi/redis/util/RedisUtils.java
@@ -21,6 +21,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.redis.RedisType;
@@ -64,7 +65,7 @@ public class RedisUtils {
                     "In a clustered instance this value will be the comma-separated list of cluster masters, such as host1:port,host2:port,host3:port.")
             .required(true)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor DATABASE = new PropertyDescriptor.Builder()
@@ -74,7 +75,7 @@ public class RedisUtils {
                     "See the databases property in redis.conf, by default databases 0-15 will be available.")
             .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
             .defaultValue("0")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .required(true)
             .build();
 
@@ -101,7 +102,7 @@ public class RedisUtils {
             .displayName("Sentinel Master")
             .description("The name of the sentinel master, require when Mode is set to Sentinel")
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
@@ -109,7 +110,7 @@ public class RedisUtils {
             .displayName("Password")
             .description("The password used to authenticate to the Redis server. See the requirepass property in redis.conf.")
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .sensitive(true)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaRegistry.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaRegistry.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaRegistry.java
index 1b7ee8b..5869278 100644
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaRegistry.java
+++ b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaRegistry.java
@@ -25,6 +25,7 @@ import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ControllerServiceInitializationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.schema.access.SchemaField;
@@ -148,7 +149,7 @@ public class AvroSchemaRegistry extends AbstractControllerService implements Sch
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .dynamic(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-rethinkdb-bundle/nifi-rethinkdb-processors/src/main/java/org/apache/nifi/processors/rethinkdb/AbstractRethinkDBProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-rethinkdb-bundle/nifi-rethinkdb-processors/src/main/java/org/apache/nifi/processors/rethinkdb/AbstractRethinkDBProcessor.java b/nifi-nar-bundles/nifi-rethinkdb-bundle/nifi-rethinkdb-processors/src/main/java/org/apache/nifi/processors/rethinkdb/AbstractRethinkDBProcessor.java
index 61ad457..4f69a2b 100644
--- a/nifi-nar-bundles/nifi-rethinkdb-bundle/nifi-rethinkdb-processors/src/main/java/org/apache/nifi/processors/rethinkdb/AbstractRethinkDBProcessor.java
+++ b/nifi-nar-bundles/nifi-rethinkdb-bundle/nifi-rethinkdb-processors/src/main/java/org/apache/nifi/processors/rethinkdb/AbstractRethinkDBProcessor.java
@@ -21,6 +21,7 @@ import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.Relationship;
@@ -40,7 +41,7 @@ abstract class AbstractRethinkDBProcessor extends AbstractProcessor {
             .description("Specifies the character set of the document data.")
             .required(true)
             .defaultValue("UTF-8")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
             .build();
 
@@ -111,7 +112,7 @@ abstract class AbstractRethinkDBProcessor extends AbstractProcessor {
                     "for determining RethinkDB key for the Flow File content")
                 .required(true)
                 .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(ResultType.STRING, true))
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
                 .build();
 
     public static AllowableValue DURABILITY_SOFT = new AllowableValue("soft", "Soft", "Don't save changes to disk before ack");
@@ -125,7 +126,7 @@ abstract class AbstractRethinkDBProcessor extends AbstractProcessor {
                 .required(true)
                 .defaultValue("hard")
                 .allowableValues(DURABILITY_HARD, DURABILITY_SOFT)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
                 .build();
 
     static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-rethinkdb-bundle/nifi-rethinkdb-processors/src/main/java/org/apache/nifi/processors/rethinkdb/DeleteRethinkDB.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-rethinkdb-bundle/nifi-rethinkdb-processors/src/main/java/org/apache/nifi/processors/rethinkdb/DeleteRethinkDB.java b/nifi-nar-bundles/nifi-rethinkdb-bundle/nifi-rethinkdb-processors/src/main/java/org/apache/nifi/processors/rethinkdb/DeleteRethinkDB.java
index 6fed9e2..0140da9 100644
--- a/nifi-nar-bundles/nifi-rethinkdb-bundle/nifi-rethinkdb-processors/src/main/java/org/apache/nifi/processors/rethinkdb/DeleteRethinkDB.java
+++ b/nifi-nar-bundles/nifi-rethinkdb-bundle/nifi-rethinkdb-processors/src/main/java/org/apache/nifi/processors/rethinkdb/DeleteRethinkDB.java
@@ -28,6 +28,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -72,7 +73,7 @@ public class DeleteRethinkDB extends AbstractRethinkDBProcessor {
             .required(true)
             .defaultValue(RETURN_CHANGES_TRUE.getValue())
             .allowableValues(RETURN_CHANGES_TRUE, RETURN_CHANGES_FALSE)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     private static final Set<Relationship> relationships;

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-rethinkdb-bundle/nifi-rethinkdb-processors/src/main/java/org/apache/nifi/processors/rethinkdb/GetRethinkDB.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-rethinkdb-bundle/nifi-rethinkdb-processors/src/main/java/org/apache/nifi/processors/rethinkdb/GetRethinkDB.java b/nifi-nar-bundles/nifi-rethinkdb-bundle/nifi-rethinkdb-processors/src/main/java/org/apache/nifi/processors/rethinkdb/GetRethinkDB.java
index aaf1046..27d8a6f 100644
--- a/nifi-nar-bundles/nifi-rethinkdb-bundle/nifi-rethinkdb-processors/src/main/java/org/apache/nifi/processors/rethinkdb/GetRethinkDB.java
+++ b/nifi-nar-bundles/nifi-rethinkdb-bundle/nifi-rethinkdb-processors/src/main/java/org/apache/nifi/processors/rethinkdb/GetRethinkDB.java
@@ -28,6 +28,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.ProcessContext;
@@ -67,7 +68,7 @@ public class GetRethinkDB extends AbstractRethinkDBProcessor {
             .required(true)
             .defaultValue(READ_MODE_SINGLE.getValue())
             .allowableValues(READ_MODE_SINGLE, READ_MODE_MAJORITY, READ_MODE_OUTDATED)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     protected String READ_MODE_KEY = "read_mode";

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-rethinkdb-bundle/nifi-rethinkdb-processors/src/main/java/org/apache/nifi/processors/rethinkdb/PutRethinkDB.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-rethinkdb-bundle/nifi-rethinkdb-processors/src/main/java/org/apache/nifi/processors/rethinkdb/PutRethinkDB.java b/nifi-nar-bundles/nifi-rethinkdb-bundle/nifi-rethinkdb-processors/src/main/java/org/apache/nifi/processors/rethinkdb/PutRethinkDB.java
index ac57957..4d3e84d 100644
--- a/nifi-nar-bundles/nifi-rethinkdb-bundle/nifi-rethinkdb-processors/src/main/java/org/apache/nifi/processors/rethinkdb/PutRethinkDB.java
+++ b/nifi-nar-bundles/nifi-rethinkdb-bundle/nifi-rethinkdb-processors/src/main/java/org/apache/nifi/processors/rethinkdb/PutRethinkDB.java
@@ -28,6 +28,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.ProcessContext;
@@ -77,7 +78,7 @@ public class PutRethinkDB extends AbstractRethinkDBProcessor {
             .required(true)
             .defaultValue(CONFLICT_STRATEGY_UPDATE.getValue())
             .allowableValues(CONFLICT_STRATEGY_UPDATE, CONFLICT_STRATEGY_REPLACE, CONFLICT_STRATEGY_ERROR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     private static final Set<Relationship> relationships;

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-riemann-bundle/nifi-riemann-processors/src/main/java/org/apache/nifi/processors/riemann/PutRiemann.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-riemann-bundle/nifi-riemann-processors/src/main/java/org/apache/nifi/processors/riemann/PutRiemann.java b/nifi-nar-bundles/nifi-riemann-bundle/nifi-riemann-processors/src/main/java/org/apache/nifi/processors/riemann/PutRiemann.java
index 6ba2ae0..1b5b5d3 100644
--- a/nifi-nar-bundles/nifi-riemann-bundle/nifi-riemann-processors/src/main/java/org/apache/nifi/processors/riemann/PutRiemann.java
+++ b/nifi-nar-bundles/nifi-riemann-bundle/nifi-riemann-processors/src/main/java/org/apache/nifi/processors/riemann/PutRiemann.java
@@ -31,6 +31,7 @@ import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -48,7 +49,7 @@ import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
 @Tags({"riemann", "monitoring", "metrics"})
-@DynamicProperty(name = "Custom Event Attribute", supportsExpressionLanguage = true,
+@DynamicProperty(name = "Custom Event Attribute", expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
   description = "These values will be attached to the Riemann event as a custom attribute",
   value = "Any value or expression")
 @CapabilityDescription("Send events to Riemann (http://riemann.io) when FlowFiles pass through this processor. " +
@@ -112,7 +113,7 @@ public class PutRiemann extends AbstractProcessor {
     .name("Service")
     .description("Name of service associated to this event (e.g. FTP File Fetched)")
     .required(false)
-    .expressionLanguageSupported(true)
+    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
     .addValidator(Validator.VALID)
     .build();
 
@@ -120,7 +121,7 @@ public class PutRiemann extends AbstractProcessor {
     .name("State")
     .description("State of service associated to this event in string form (e.g. ok, warning, foo)")
     .required(false)
-    .expressionLanguageSupported(true)
+    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
     .addValidator(Validator.VALID)
     .build();
 
@@ -128,9 +129,8 @@ public class PutRiemann extends AbstractProcessor {
     .name("Time")
     .description("Time of event in unix epoch seconds (long), default: (current time)")
     .required(false)
-    .expressionLanguageSupported(true)
+    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
     .addValidator(Validator.VALID)
-    .expressionLanguageSupported(true)
     .build();
 
   public static final PropertyDescriptor ATTR_HOST = new PropertyDescriptor.Builder()
@@ -138,7 +138,7 @@ public class PutRiemann extends AbstractProcessor {
     .description("A hostname associated to this event (e.g. nifi-app1)")
     .required(false)
     .defaultValue("${hostname()}")
-    .expressionLanguageSupported(true)
+    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
     .addValidator(Validator.VALID)
     .build();
 
@@ -147,7 +147,7 @@ public class PutRiemann extends AbstractProcessor {
     .description("Floating point value in seconds until Riemann considers this event as \"expired\"")
     .required(false)
     .addValidator(Validator.VALID)
-    .expressionLanguageSupported(true)
+    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
     .build();
 
   public static final PropertyDescriptor ATTR_METRIC = new PropertyDescriptor.Builder()
@@ -155,14 +155,14 @@ public class PutRiemann extends AbstractProcessor {
     .description("Floating point number associated to this event")
     .required(false)
     .addValidator(Validator.VALID)
-    .expressionLanguageSupported(true)
+    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
     .build();
 
   public static final PropertyDescriptor ATTR_DESCRIPTION = new PropertyDescriptor.Builder()
     .name("Description")
     .description("Description associated to the event")
     .required(false)
-    .expressionLanguageSupported(true)
+    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
     .addValidator(Validator.VALID)
     .build();
 
@@ -171,7 +171,7 @@ public class PutRiemann extends AbstractProcessor {
     .name("Tags")
     .description("Comma separated list of tags associated to the event")
     .required(false)
-    .expressionLanguageSupported(true)
+    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
     .addValidator(Validator.VALID)
     .build();
 
@@ -223,7 +223,7 @@ public class PutRiemann extends AbstractProcessor {
   protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
     return new PropertyDescriptor.Builder()
       .name(propertyDescriptorName)
-      .expressionLanguageSupported(true)
+      .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
       .addValidator(Validator.VALID)
       .required(false)
       .dynamic(true)

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/lookup/script/ScriptedLookupService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/lookup/script/ScriptedLookupService.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/lookup/script/ScriptedLookupService.java
index c3b6c9d..25ac39a 100644
--- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/lookup/script/ScriptedLookupService.java
+++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/lookup/script/ScriptedLookupService.java
@@ -30,6 +30,7 @@ import org.apache.nifi.components.state.StateManager;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.ControllerServiceInitializationContext;
 import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.lookup.LookupFailureException;
 import org.apache.nifi.lookup.LookupService;
@@ -154,7 +155,7 @@ public class ScriptedLookupService extends AbstractScriptedControllerService imp
                 .name(propertyDescriptorName)
                 .required(false)
                 .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
                 .dynamic(true)
                 .build();
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ExecuteScript.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ExecuteScript.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ExecuteScript.java
index b0b61a2..9b241f3 100644
--- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ExecuteScript.java
+++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ExecuteScript.java
@@ -32,6 +32,7 @@ import org.apache.nifi.components.RequiredPermission;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -66,7 +67,7 @@ import java.util.Set;
 @DynamicProperty(
         name = "A script engine property to update",
         value = "The value to set it to",
-        supportsExpressionLanguage = true,
+        expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
         description = "Updates a script engine property specified by the Dynamic Property's key with the value "
                 + "specified by the Dynamic Property's value")
 @Restricted(
@@ -133,7 +134,7 @@ public class ExecuteScript extends AbstractSessionFactoryProcessor {
                 .name(propertyDescriptorName)
                 .required(false)
                 .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
                 .dynamic(true)
                 .build();
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/InvokeScriptedProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/InvokeScriptedProcessor.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/InvokeScriptedProcessor.java
index 032be32..5805d85 100644
--- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/InvokeScriptedProcessor.java
+++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/InvokeScriptedProcessor.java
@@ -34,6 +34,7 @@ import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.controller.ControllerServiceLookup;
 import org.apache.nifi.controller.NodeTypeProvider;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -67,7 +68,8 @@ import java.util.concurrent.atomic.AtomicReference;
         + "the class. Processor methods such as onTrigger() will be delegated to the scripted Processor instance. Also any "
         + "Relationships or PropertyDescriptors defined by the scripted processor will be added to the configuration dialog.  "
         + "Experimental: Impact of sustained usage not yet verified.")
-@DynamicProperty(name = "A script engine property to update", value = "The value to set it to", supportsExpressionLanguage = true,
+@DynamicProperty(name = "A script engine property to update", value = "The value to set it to",
+        expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
         description = "Updates a script engine property specified by the Dynamic Property's key with the value specified by the Dynamic Property's value")
 @Stateful(scopes = {Scope.LOCAL, Scope.CLUSTER},
         description = "Scripts can store and retrieve state using the State Management APIs. Consult the State Manager section of the Developer's Guide for more details.")
@@ -185,7 +187,7 @@ public class InvokeScriptedProcessor extends AbstractSessionFactoryProcessor {
                 .name(propertyDescriptorName)
                 .required(false)
                 .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
                 .dynamic(true)
                 .build();
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/reporting/script/ScriptedReportingTask.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/reporting/script/ScriptedReportingTask.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/reporting/script/ScriptedReportingTask.java
index 111839d..fb1d737 100644
--- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/reporting/script/ScriptedReportingTask.java
+++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/reporting/script/ScriptedReportingTask.java
@@ -29,6 +29,7 @@ import org.apache.nifi.components.RequiredPermission;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
@@ -60,7 +61,7 @@ import java.util.Map;
 @DynamicProperty(
         name = "A script engine property to update",
         value = "The value to set it to",
-        supportsExpressionLanguage = true,
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
         description = "Updates a script engine property specified by the Dynamic Property's key with the value "
                 + "specified by the Dynamic Property's value")
 @Restricted(
@@ -107,7 +108,7 @@ public class ScriptedReportingTask extends AbstractReportingTask {
                 .name(propertyDescriptorName)
                 .required(false)
                 .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
                 .dynamic(true)
                 .build();
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/AbstractScriptedControllerService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/AbstractScriptedControllerService.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/AbstractScriptedControllerService.java
index 126bba3..ee25d55 100644
--- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/AbstractScriptedControllerService.java
+++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/AbstractScriptedControllerService.java
@@ -22,6 +22,7 @@ import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.util.StandardValidators;
 
@@ -86,7 +87,7 @@ public abstract class AbstractScriptedControllerService extends AbstractControll
                 .name(propertyDescriptorName)
                 .required(false)
                 .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
                 .dynamic(true)
                 .build();
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptingComponentHelper.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptingComponentHelper.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptingComponentHelper.java
index e838baa..776fc49 100644
--- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptingComponentHelper.java
+++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptingComponentHelper.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.script;
 
 import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 
 import java.io.File;
@@ -175,7 +176,7 @@ public class ScriptingComponentHelper {
                     .allowableValues(engines)
                     .defaultValue(engines[0].getValue())
                     .required(true)
-                    .expressionLanguageSupported(false)
+                    .expressionLanguageSupported(ExpressionLanguageScope.NONE)
                     .build();
             descriptors.add(SCRIPT_ENGINE);
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptingComponentUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptingComponentUtils.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptingComponentUtils.java
index e3af457..5b7b63b 100644
--- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptingComponentUtils.java
+++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptingComponentUtils.java
@@ -18,6 +18,7 @@ package org.apache.nifi.script;
 
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.util.StandardValidators;
 
@@ -43,7 +44,7 @@ public class ScriptingComponentUtils {
             .required(false)
             .description("Path to script file to execute. Only one of Script File or Script Body may be used")
             .addValidator(new StandardValidators.FileExistsValidator(true))
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     /** A property descriptor for specifying the body of a script */
@@ -52,7 +53,7 @@ public class ScriptingComponentUtils {
             .required(false)
             .description("Body of script to execute. Only one of Script File or Script Body may be used")
             .addValidator(Validator.VALID)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
 
     /** A property descriptor for specifying the location of additional modules to be used by the script */
@@ -60,7 +61,7 @@ public class ScriptingComponentUtils {
             .name("Module Directory")
             .description("Comma-separated list of paths to files and/or directories which contain modules required by the script.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/AbstractSiteToSiteReportingTask.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/AbstractSiteToSiteReportingTask.java b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/AbstractSiteToSiteReportingTask.java
index 28106a6..341a6d8 100644
--- a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/AbstractSiteToSiteReportingTask.java
+++ b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/AbstractSiteToSiteReportingTask.java
@@ -24,6 +24,7 @@ import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.events.EventReporter;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.remote.client.SiteToSiteClient;
@@ -52,7 +53,7 @@ public abstract class AbstractSiteToSiteReportingTask extends AbstractReportingT
             .description("The URL of the destination NiFi instance to send data to, " +
                     "should be a comma-separated list of address in the format of http(s)://host:port/nifi.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(new NiFiUrlValidator())
             .build();
     static final PropertyDescriptor PORT_NAME = new PropertyDescriptor.Builder()
@@ -60,7 +61,7 @@ public abstract class AbstractSiteToSiteReportingTask extends AbstractReportingT
             .displayName("Input Port Name")
             .description("The name of the Input Port to deliver data to.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
     static final PropertyDescriptor SSL_CONTEXT = new PropertyDescriptor.Builder()
@@ -75,7 +76,7 @@ public abstract class AbstractSiteToSiteReportingTask extends AbstractReportingT
             .displayName("Instance URL")
             .description("The URL of this instance to use in the Content URI of each event.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue("http://${hostname(true)}:8080/nifi")
             .addValidator(new NiFiUrlValidator())
             .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteBulletinReportingTask.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteBulletinReportingTask.java b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteBulletinReportingTask.java
index 102a5d3..fac7696 100644
--- a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteBulletinReportingTask.java
+++ b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteBulletinReportingTask.java
@@ -26,6 +26,7 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.RequiredPermission;
 import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.remote.Transaction;
@@ -74,7 +75,7 @@ public class SiteToSiteBulletinReportingTask extends AbstractSiteToSiteReporting
         .name("Platform")
         .description("The value to use for the platform field in each provenance event.")
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .defaultValue("nifi")
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteProvenanceReportingTask.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteProvenanceReportingTask.java b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteProvenanceReportingTask.java
index 22ecfc5..6db30b8 100644
--- a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteProvenanceReportingTask.java
+++ b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteProvenanceReportingTask.java
@@ -31,6 +31,7 @@ import org.apache.nifi.components.RequiredPermission;
 import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.status.ProcessGroupStatus;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.provenance.ProvenanceEventRecord;
@@ -88,7 +89,7 @@ public class SiteToSiteProvenanceReportingTask extends AbstractSiteToSiteReporti
         .displayName("Platform")
         .description("The value to use for the platform field in each provenance event.")
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .defaultValue("nifi")
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteStatusReportingTask.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteStatusReportingTask.java b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteStatusReportingTask.java
index 526b5d5..75e9811 100644
--- a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteStatusReportingTask.java
+++ b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteStatusReportingTask.java
@@ -49,6 +49,7 @@ import org.apache.nifi.controller.status.PortStatus;
 import org.apache.nifi.controller.status.ProcessGroupStatus;
 import org.apache.nifi.controller.status.ProcessorStatus;
 import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.remote.Transaction;
@@ -66,7 +67,7 @@ public class SiteToSiteStatusReportingTask extends AbstractSiteToSiteReportingTa
         .name("Platform")
         .description("The value to use for the platform field in each status record.")
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .defaultValue("nifi")
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .build();
@@ -75,7 +76,7 @@ public class SiteToSiteStatusReportingTask extends AbstractSiteToSiteReportingTa
         .description("A regex specifying which component types to report.  Any component type matching this regex will be included.  "
                 + "Component types are: Processor, RootProcessGroup, ProcessGroup, RemoteProcessGroup, Connection, InputPort, OutputPort")
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .defaultValue("(Processor|ProcessGroup|RemoteProcessGroup|RootProcessGroup|Connection|InputPort|OutputPort)")
         .addValidator(StandardValidators.createRegexValidator(0, Integer.MAX_VALUE, true))
         .build();
@@ -83,7 +84,7 @@ public class SiteToSiteStatusReportingTask extends AbstractSiteToSiteReportingTa
         .name("Component Name Filter Regex")
         .description("A regex specifying which component names to report.  Any component name matching this regex will be included.")
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .defaultValue(".*")
         .addValidator(StandardValidators.createRegexValidator(0, Integer.MAX_VALUE, true))
         .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-processors/src/main/java/org/apache/nifi/processors/slack/PutSlack.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-processors/src/main/java/org/apache/nifi/processors/slack/PutSlack.java b/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-processors/src/main/java/org/apache/nifi/processors/slack/PutSlack.java
index d91a52e..fd4a302 100644
--- a/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-processors/src/main/java/org/apache/nifi/processors/slack/PutSlack.java
+++ b/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-processors/src/main/java/org/apache/nifi/processors/slack/PutSlack.java
@@ -26,6 +26,7 @@ import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
 import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -61,7 +62,7 @@ import java.util.TreeSet;
 @CapabilityDescription("Sends a message to your team on slack.com")
 @InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
 @DynamicProperty(name = "A JSON object to add to Slack's \"attachments\" JSON payload.", value = "JSON-formatted string to add to Slack's payload JSON appended to the \"attachments\" JSON array.",
-        supportsExpressionLanguage = true,
+        expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
         description = "Converts the contents of each value specified by the Dynamic Property's value to JSON and appends it to the payload being sent to Slack.")
 public class PutSlack extends AbstractProcessor {
 
@@ -82,7 +83,7 @@ public class PutSlack extends AbstractProcessor {
             .displayName("Webhook Text")
             .description("The text sent in the webhook message")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -93,7 +94,7 @@ public class PutSlack extends AbstractProcessor {
             .description("A public channel using #channel or direct message using @username. If not specified, " +
                     "the default webhook channel as specified in Slack's Incoming Webhooks web interface is used.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -103,7 +104,7 @@ public class PutSlack extends AbstractProcessor {
             .displayName("Username")
             .description("The displayed Slack username")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -113,7 +114,7 @@ public class PutSlack extends AbstractProcessor {
             .displayName("Icon URL")
             .description("Icon URL to be used for the message")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.URL_VALIDATOR)
             .build();
 
@@ -123,7 +124,7 @@ public class PutSlack extends AbstractProcessor {
             .displayName("Icon Emoji")
             .description("Icon Emoji to be used for the message. Must begin and end with a colon, e.g. :ghost:")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .addValidator(new EmojiValidator())
             .build();
@@ -163,7 +164,7 @@ public class PutSlack extends AbstractProcessor {
                 .required(false)
                 .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
                 .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
                 .dynamic(true)
                 .build();
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/GetSolr.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/GetSolr.java b/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/GetSolr.java
index 3435ce4..6260304 100755
--- a/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/GetSolr.java
+++ b/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/GetSolr.java
@@ -46,6 +46,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.logging.ComponentLog;
@@ -110,7 +111,7 @@ public class GetSolr extends SolrProcessor {
             .displayName("Record Writer")
             .description("The Record Writer to use in order to write Solr documents to FlowFiles. Must be set if \"Records\" is used as return type.")
             .identifiesControllerService(RecordSetWriterFactory.class)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(false)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/PutSolrContentStream.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/PutSolrContentStream.java b/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/PutSolrContentStream.java
index 4a16953..71f186b 100755
--- a/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/PutSolrContentStream.java
+++ b/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/PutSolrContentStream.java
@@ -24,6 +24,7 @@ import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -84,7 +85,7 @@ public class PutSolrContentStream extends SolrProcessor {
             .description("The path in Solr to post the ContentStream")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("/update/json/docs")
             .build();
 
@@ -93,7 +94,7 @@ public class PutSolrContentStream extends SolrProcessor {
             .description("Content-Type being sent to Solr")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("application/json")
             .build();
 
@@ -102,7 +103,7 @@ public class PutSolrContentStream extends SolrProcessor {
             .description("The number of milliseconds before the given update is committed")
             .required(false)
             .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("5000")
             .build();
 
@@ -175,13 +176,13 @@ public class PutSolrContentStream extends SolrProcessor {
                 .name(propertyDescriptorName)
                 .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
                 .dynamic(true)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
                 .build();
     }
 
     @Override
     public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
-        FlowFile flowFile = session.get();
+        final FlowFile flowFile = session.get();
         if ( flowFile == null ) {
             return;
         }
@@ -235,7 +236,7 @@ public class PutSolrContentStream extends SolrProcessor {
 
                         @Override
                         public String getContentType() {
-                            return context.getProperty(CONTENT_TYPE).evaluateAttributeExpressions().getValue();
+                            return context.getProperty(CONTENT_TYPE).evaluateAttributeExpressions(flowFile).getValue();
                         }
                     });
 
@@ -263,7 +264,7 @@ public class PutSolrContentStream extends SolrProcessor {
         } else if (connectionError.get() != null) {
             getLogger().error("Failed to send {} to Solr due to {}; routing to connection_failure",
                     new Object[]{flowFile, connectionError.get()});
-            flowFile = session.penalize(flowFile);
+            session.penalize(flowFile);
             session.transfer(flowFile, REL_CONNECTION_FAILURE);
         } else {
             StringBuilder transitUri = new StringBuilder("solr://");

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/SolrUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/SolrUtils.java b/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/SolrUtils.java
index 8977b7c..6a7e438 100755
--- a/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/SolrUtils.java
+++ b/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/SolrUtils.java
@@ -27,6 +27,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.context.PropertyContext;
 import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.io.OutputStreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.serialization.record.ListRecordSet;
@@ -79,7 +80,7 @@ public class SolrUtils {
             .description("The Solr collection name, only used with a Solr Type of Cloud")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor SOLR_LOCATION = new PropertyDescriptor
@@ -89,7 +90,7 @@ public class SolrUtils {
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING))
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor BASIC_USERNAME = new PropertyDescriptor
@@ -98,7 +99,7 @@ public class SolrUtils {
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING))
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor BASIC_PASSWORD = new PropertyDescriptor
@@ -107,7 +108,7 @@ public class SolrUtils {
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING))
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .sensitive(true)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-controller-service/src/main/java/org/apache/nifi/controller/livy/LivySessionController.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-controller-service/src/main/java/org/apache/nifi/controller/livy/LivySessionController.java b/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-controller-service/src/main/java/org/apache/nifi/controller/livy/LivySessionController.java
index 8a512f8..f9ded28 100644
--- a/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-controller-service/src/main/java/org/apache/nifi/controller/livy/LivySessionController.java
+++ b/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-controller-service/src/main/java/org/apache/nifi/controller/livy/LivySessionController.java
@@ -62,6 +62,7 @@ import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 
 import org.apache.nifi.controller.api.livy.LivySessionService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 
 import javax.net.ssl.HttpsURLConnection;
 import javax.net.ssl.KeyManagerFactory;
@@ -79,7 +80,7 @@ public class LivySessionController extends AbstractControllerService implements
             .description("The hostname (or IP address) of the Livy server.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor LIVY_PORT = new PropertyDescriptor.Builder()
@@ -88,7 +89,7 @@ public class LivySessionController extends AbstractControllerService implements
             .description("The port number for the Livy server.")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue("8998")
             .build();
 
@@ -99,7 +100,7 @@ public class LivySessionController extends AbstractControllerService implements
             .required(true)
             .defaultValue("2")
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor SESSION_TYPE = new PropertyDescriptor.Builder()
@@ -119,7 +120,7 @@ public class LivySessionController extends AbstractControllerService implements
             .required(true)
             .defaultValue("2 sec")
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor JARS = new PropertyDescriptor.Builder()
@@ -128,7 +129,7 @@ public class LivySessionController extends AbstractControllerService implements
             .description("JARs to be used in the Spark session.")
             .required(false)
             .addValidator(StandardValidators.createListValidator(true, true, StandardValidators.FILE_EXISTS_VALIDATOR))
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
     public static final PropertyDescriptor FILES = new PropertyDescriptor.Builder()
@@ -137,6 +138,7 @@ public class LivySessionController extends AbstractControllerService implements
             .description("Files to be used in the Spark session.")
             .required(false)
             .addValidator(StandardValidators.createListValidator(true, true, StandardValidators.FILE_EXISTS_VALIDATOR))
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue(null)
             .build();
 
@@ -192,13 +194,11 @@ public class LivySessionController extends AbstractControllerService implements
 
     @OnEnabled
     public void onConfigured(final ConfigurationContext context) {
-        ComponentLog log = getLogger();
-
         final String livyHost = context.getProperty(LIVY_HOST).evaluateAttributeExpressions().getValue();
         final String livyPort = context.getProperty(LIVY_PORT).evaluateAttributeExpressions().getValue();
         final String sessionPoolSize = context.getProperty(SESSION_POOL_SIZE).evaluateAttributeExpressions().getValue();
         final String sessionKind = context.getProperty(SESSION_TYPE).getValue();
-        final long sessionManagerStatusInterval = context.getProperty(SESSION_MGR_STATUS_INTERVAL).asTimePeriod(TimeUnit.MILLISECONDS);
+        final long sessionManagerStatusInterval = context.getProperty(SESSION_MGR_STATUS_INTERVAL).evaluateAttributeExpressions().asTimePeriod(TimeUnit.MILLISECONDS);
         final String jars = context.getProperty(JARS).evaluateAttributeExpressions().getValue();
         final String files = context.getProperty(FILES).evaluateAttributeExpressions().getValue();
         sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
@@ -439,7 +439,6 @@ public class LivySessionController extends AbstractControllerService implements
     }
 
     private JSONObject readJSONObjectFromUrlPOST(String urlString, Map<String, String> headers, String payload) throws IOException, JSONException {
-        URL url = new URL(urlString);
         HttpURLConnection connection = getConnection(urlString);
 
         connection.setRequestMethod(POST);

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-processors/src/main/java/org/apache/nifi/processors/livy/ExecuteSparkInteractive.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-processors/src/main/java/org/apache/nifi/processors/livy/ExecuteSparkInteractive.java b/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-processors/src/main/java/org/apache/nifi/processors/livy/ExecuteSparkInteractive.java
index 59deb36..dcb6b82 100644
--- a/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-processors/src/main/java/org/apache/nifi/processors/livy/ExecuteSparkInteractive.java
+++ b/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-processors/src/main/java/org/apache/nifi/processors/livy/ExecuteSparkInteractive.java
@@ -56,6 +56,7 @@ import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 
 import org.apache.nifi.controller.api.livy.LivySessionService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 
 @InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
 @Tags({"spark", "livy", "http", "execute"})
@@ -80,7 +81,7 @@ public class ExecuteSparkInteractive extends AbstractProcessor {
                     + "Language is not evaluated for flow file contents.")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     /**
@@ -94,7 +95,7 @@ public class ExecuteSparkInteractive extends AbstractProcessor {
             .required(true)
             .defaultValue("UTF-8")
             .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor STATUS_CHECK_INTERVAL = new PropertyDescriptor.Builder()
@@ -103,7 +104,7 @@ public class ExecuteSparkInteractive extends AbstractProcessor {
             .description("The amount of time to wait between checking the status of an operation.")
             .required(true)
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("1 sec")
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractDatabaseFetchProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractDatabaseFetchProcessor.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractDatabaseFetchProcessor.java
index 70cb656..8c22fed 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractDatabaseFetchProcessor.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractDatabaseFetchProcessor.java
@@ -22,6 +22,7 @@ import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.dbcp.DBCPService;
 import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -108,7 +109,7 @@ public abstract class AbstractDatabaseFetchProcessor extends AbstractSessionFact
             .description("The name of the database table to be queried.")
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor COLUMN_NAMES = new PropertyDescriptor.Builder()
@@ -119,7 +120,7 @@ public abstract class AbstractDatabaseFetchProcessor extends AbstractSessionFact
                     + "to use consistent column names for a given table for incremental fetch to work properly.")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor MAX_VALUE_COLUMN_NAMES = new PropertyDescriptor.Builder()
@@ -135,7 +136,7 @@ public abstract class AbstractDatabaseFetchProcessor extends AbstractSessionFact
                     + "to use consistent max-value column names for a given table for incremental fetch to work properly.")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor QUERY_TIMEOUT = new PropertyDescriptor.Builder()
@@ -145,7 +146,7 @@ public abstract class AbstractDatabaseFetchProcessor extends AbstractSessionFact
             .defaultValue("0 seconds")
             .required(true)
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor WHERE_CLAUSE = new PropertyDescriptor.Builder()
@@ -153,7 +154,7 @@ public abstract class AbstractDatabaseFetchProcessor extends AbstractSessionFact
             .displayName("Additional WHERE clause")
             .description("A custom clause to be added in the WHERE condition when generating SQL requests.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -213,7 +214,7 @@ public abstract class AbstractDatabaseFetchProcessor extends AbstractSessionFact
                 .required(false)
                 .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
                 .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
                 .dynamic(true)
                 .build();
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ControlRate.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ControlRate.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ControlRate.java
index cf2364a..4634f04 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ControlRate.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ControlRate.java
@@ -43,6 +43,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -101,7 +102,7 @@ public class ControlRate extends AbstractProcessor {
                     + "This value is ignored if Rate Control Criteria is not set to 'attribute value'. Changing this value resets the rate counters.")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
     public static final PropertyDescriptor TIME_PERIOD = new PropertyDescriptor.Builder()
             .name("Time Duration")
@@ -116,7 +117,7 @@ public class ControlRate extends AbstractProcessor {
                     + "each value specified by the attribute with this name. Changing this value resets the rate counters.")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
 
     public static final Relationship REL_SUCCESS = new Relationship.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertCharacterSet.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertCharacterSet.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertCharacterSet.java
index 50ac358..ff6964b 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertCharacterSet.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertCharacterSet.java
@@ -30,6 +30,7 @@ import org.apache.nifi.processor.ProcessorInitializationContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
@@ -88,14 +89,14 @@ public class ConvertCharacterSet extends AbstractProcessor {
     public static final PropertyDescriptor INPUT_CHARSET = new PropertyDescriptor.Builder()
             .name("Input Character Set")
             .description("The name of the CharacterSet to expect for Input")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
             .required(true)
             .build();
     public static final PropertyDescriptor OUTPUT_CHARSET = new PropertyDescriptor.Builder()
             .name("Output Character Set")
             .description("The name of the CharacterSet to convert to")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
             .required(true)
             .build();


[14/15] nifi git commit: NIFI-4149 - Indicate if EL is evaluated against FFs or not - take into account input requirement for documentation rendering - Renamed variable registry scope and added comments - Doc + change in mock framework to check scope + u

Posted by ma...@apache.org.
NIFI-4149 - Indicate if EL is evaluated against FFs or not
- take into account input requirement for documentation rendering
- Renamed variable registry scope and added comments
- Doc + change in mock framework to check scope + update of components + UI


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/4c787799
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/4c787799
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/4c787799

Branch: refs/heads/master
Commit: 4c787799ff7d971eb924df1e496da8338e6ab192
Parents: 7ff38f6
Author: Pierre Villard <pi...@gmail.com>
Authored: Tue Sep 19 09:27:46 2017 +0200
Committer: Mark Payne <ma...@hotmail.com>
Committed: Fri Apr 6 12:40:54 2018 -0400

----------------------------------------------------------------------
 .../annotation/behavior/DynamicProperty.java    |   5 +
 .../nifi/components/PropertyDescriptor.java     |  31 ++-
 .../expression/ExpressionLanguageScope.java     |  68 ++++++
 .../email/EmailNotificationService.java         |  25 ++-
 .../http/HttpNotificationService.java           |   9 +-
 .../asciidoc/expression-language-guide.adoc     |  14 +-
 .../nifi/util/MockControllerServiceLookup.java  |   5 +
 .../apache/nifi/util/MockProcessContext.java    |  11 +-
 .../org/apache/nifi/util/MockPropertyValue.java |  65 +++++-
 .../apache/nifi/util/MockValidationContext.java |   2 +-
 .../reporting/ambari/AmbariReportingTask.java   |   9 +-
 .../nifi/amqp/processors/PublishAMQP.java       |   5 +-
 .../atlas/reporting/ReportLineageToAtlas.java   |  87 +++----
 .../atlas/resolver/RegexClusterResolver.java    |   3 +-
 .../processors/aws/AbstractAWSProcessor.java    |  14 +-
 .../factory/CredentialPropertyDescriptors.java  |  23 +-
 .../aws/dynamodb/AbstractDynamoDBProcessor.java |  17 +-
 .../AbstractKinesisFirehoseProcessor.java       |   4 +-
 .../stream/AbstractKinesisStreamProcessor.java  |   3 +-
 .../aws/lambda/AbstractAWSLambdaProcessor.java  |   5 +-
 .../processors/aws/s3/AbstractS3Processor.java  |  19 +-
 .../aws/sns/AbstractSNSProcessor.java           |   5 +-
 .../aws/sqs/AbstractSQSProcessor.java           |   3 +-
 .../aws/cloudwatch/PutCloudWatchMetric.java     |  23 +-
 .../aws/kinesis/stream/PutKinesisStream.java    |   3 +-
 .../nifi/processors/aws/s3/DeleteS3Object.java  |   3 +-
 .../nifi/processors/aws/s3/FetchS3Object.java   |   3 +-
 .../apache/nifi/processors/aws/s3/ListS3.java   |   9 +-
 .../nifi/processors/aws/s3/PutS3Object.java     |  18 +-
 .../apache/nifi/processors/aws/sns/PutSNS.java  |   7 +-
 .../nifi/processors/aws/sqs/DeleteSQS.java      |   3 +-
 .../apache/nifi/processors/aws/sqs/GetSQS.java  |   7 +-
 .../apache/nifi/processors/aws/sqs/PutSQS.java  |   5 +-
 .../aws/cloudwatch/ITPutCloudWatchMetric.java   |   5 +-
 .../nifi/processors/aws/s3/ITPutS3Object.java   |   6 +-
 .../nifi/processors/aws/sns/ITPutSNS.java       |  10 +-
 .../nifi/processors/aws/sns/TestPutSNS.java     |  18 +-
 .../nifi/processors/aws/sqs/ITPutSQS.java       |   1 -
 .../nifi/processors/aws/sqs/TestPutSQS.java     |  16 +-
 .../azure/AbstractAzureBlobProcessor.java       |  23 +-
 .../azure/eventhub/ConsumeAzureEventHub.java    |  29 +--
 .../azure/eventhub/GetAzureEventHub.java        |  19 +-
 .../azure/eventhub/PutAzureEventHub.java        |   7 +-
 .../azure/storage/ListAzureBlobStorage.java     |  12 +-
 .../azure/storage/utils/AzureStorageUtils.java  |  33 ++-
 .../azure/storage/ITDeleteAzureBlobStorage.java |  15 +-
 .../azure/storage/ITFetchAzureBlobStorage.java  |   2 -
 .../azure/storage/ITPutAzureStorageBlob.java    |   2 -
 .../cassandra/AbstractCassandraProcessor.java   |  11 +-
 .../processors/cassandra/PutCassandraQL.java    |   3 +-
 .../processors/cassandra/QueryCassandra.java    |   7 +-
 .../mysql/processors/CaptureChangeMySQL.java    |  24 +-
 .../schemaregistry/ConfluentSchemaRegistry.java |   5 +-
 .../couchbase/AbstractCouchbaseProcessor.java   |   3 +-
 .../processors/couchbase/GetCouchbaseKey.java   |   2 +-
 .../processors/couchbase/PutCouchbaseKey.java   |   3 +-
 .../couchbase/TestCouchbaseClusterService.java  |   1 -
 .../couchbase/TestGetCouchbaseKey.java          |   1 -
 .../couchbase/TestPutCouchbaseKey.java          |   1 -
 .../reporting/datadog/DataDogReportingTask.java |   9 +-
 .../druid/DruidTranquilityController.java       |  33 +--
 .../nifi/processors/druid/PutDruidRecord.java   |   4 +-
 .../AbstractElasticsearch5Processor.java        |   5 +-
 ...tElasticsearch5TransportClientProcessor.java |  11 +-
 .../elasticsearch/DeleteElasticsearch5.java     |   7 +-
 .../elasticsearch/FetchElasticsearch5.java      |   7 +-
 .../elasticsearch/PutElasticsearch5.java        |  11 +-
 .../ITDeleteElasticsearch5Test.java             |  13 +-
 .../elasticsearch/TestDeleteElasticsearch5.java |  17 +-
 .../elasticsearch/TestFetchElasticsearch5.java  |   8 -
 .../elasticsearch/TestPutElasticsearch5.java    |  43 ++--
 .../AbstractElasticsearchHttpProcessor.java     |   9 +-
 .../AbstractElasticsearchProcessor.java         |   7 +-
 ...ctElasticsearchTransportClientProcessor.java |  12 +-
 .../elasticsearch/FetchElasticsearch.java       |   7 +-
 .../elasticsearch/FetchElasticsearchHttp.java   |  11 +-
 .../elasticsearch/PutElasticsearch.java         |  11 +-
 .../elasticsearch/PutElasticsearchHttp.java     |  13 +-
 .../PutElasticsearchHttpRecord.java             |  11 +-
 .../elasticsearch/QueryElasticsearchHttp.java   |  56 +++--
 .../elasticsearch/ScrollElasticsearchHttp.java  |  41 ++--
 .../elasticsearch/ITQueryElasticsearchHttp.java |   2 -
 .../ITScrollElasticsearchHttp.java              |   1 -
 .../elasticsearch/TestFetchElasticsearch.java   |  42 ++--
 .../TestFetchElasticsearchHttp.java             |  53 ++---
 .../elasticsearch/TestPutElasticsearch.java     |  40 ++--
 .../elasticsearch/TestPutElasticsearchHttp.java |  51 ++---
 .../TestPutElasticsearchHttpRecord.java         |  49 ++--
 .../TestQueryElasticsearchHttp.java             |  14 +-
 .../TestScrollElasticsearchHttp.java            |  12 +-
 .../email/AbstractEmailProcessor.java           |  15 +-
 .../nifi/processors/email/ConsumeEWS.java       |  17 +-
 .../processors/email/ExtractEmailHeaders.java   |   3 +-
 .../nifi/processors/email/ListenSMTP.java       |  11 +-
 .../nifi/processors/AbstractEnrichIP.java       |   3 +-
 .../enrich/AbstractEnrichProcessor.java         |   5 +-
 .../apache/nifi/hadoop/KerberosProperties.java  |   5 +-
 .../hadoop/AbstractHadoopProcessor.java         |   7 +-
 .../AbstractListenEventBatchingProcessor.java   |   3 +-
 .../util/listen/ListenerProperties.java         |   3 +-
 .../util/put/AbstractPutEventProcessor.java     |   9 +-
 .../nifi/schema/access/SchemaAccessUtils.java   |   5 +-
 .../hadoop/AbstractFetchHDFSRecord.java         |   3 +-
 .../main/java/org/apache/nifi/csv/CSVUtils.java |  27 +--
 .../nifi/serialization/DateTimeUtils.java       |   7 +-
 .../nifi/web/api/dto/PropertyDescriptorDTO.java |  15 ++
 .../html/HtmlDocumentationWriter.java           |  62 ++++-
 .../example/DeprecatedProcessor.java            |  76 ++++---
 .../example/FullyDocumentedProcessor.java       |  66 ++++--
 .../html/ProcessorDocumentationWriterTest.java  |  21 +-
 .../apache/nifi/controller/TemplateUtils.java   |   1 +
 .../controller/DummyScheduledReportingTask.java |   5 +-
 .../org/apache/nifi/web/api/dto/DtoFactory.java |   7 +
 .../src/main/webapp/js/nf/nf-common.js          |   2 +-
 .../processors/gcp/AbstractGCPProcessor.java    |   5 +-
 .../factory/CredentialPropertyDescriptors.java  |   9 +-
 .../processors/gcp/storage/DeleteGCSObject.java |   7 +-
 .../processors/gcp/storage/FetchGCSObject.java  |   9 +-
 .../processors/gcp/storage/ListGCSBucket.java   |   5 +-
 .../processors/gcp/storage/PutGCSObject.java    |  17 +-
 .../processors/groovyx/ExecuteGroovyScript.java |  18 +-
 .../nifi/processors/hadoop/DeleteHDFS.java      |   3 +-
 .../nifi/processors/hadoop/FetchHDFS.java       |   3 +-
 .../apache/nifi/processors/hadoop/MoveHDFS.java |  61 +++--
 .../apache/nifi/processors/hadoop/PutHDFS.java  |   5 +-
 .../hadoop/inotify/GetHDFSEvents.java           |   3 +-
 .../apache/nifi/hbase/AbstractDeleteHBase.java  |   5 +-
 .../org/apache/nifi/hbase/AbstractPutHBase.java |  13 +-
 .../org/apache/nifi/hbase/DeleteHBaseRow.java   |   7 +-
 .../org/apache/nifi/hbase/FetchHBaseRow.java    |   7 +-
 .../java/org/apache/nifi/hbase/GetHBase.java    |   9 +-
 .../org/apache/nifi/hbase/PutHBaseJSON.java     |   5 +-
 .../org/apache/nifi/hbase/PutHBaseRecord.java   |   7 +-
 .../java/org/apache/nifi/hbase/ScanHBase.java   |  21 +-
 .../apache/nifi/hbase/TestDeleteHBaseRow.java   |   2 -
 .../nifi/dbcp/hive/HiveConnectionPool.java      |  15 +-
 .../nifi/processors/hive/ConvertAvroToORC.java  |   3 +-
 .../apache/nifi/processors/hive/PutHiveQL.java  |   3 +-
 .../nifi/processors/hive/PutHiveStreaming.java  |  17 +-
 .../nifi/processors/hive/SelectHiveQL.java      |  15 +-
 .../processors/hl7/ExtractHL7Attributes.java    |   3 +-
 .../apache/nifi/processors/hl7/RouteHL7.java    |   3 +-
 .../org/apache/nifi/AbstractHTMLProcessor.java  |   5 +-
 .../java/org/apache/nifi/GetHTMLElement.java    |   8 +-
 .../java/org/apache/nifi/ModifyHTMLElement.java |   5 +-
 .../java/org/apache/nifi/PutHTMLElement.java    |   5 +-
 .../cache/AbstractIgniteCacheProcessor.java     |   3 +-
 .../influxdb/AbstractInfluxDBProcessor.java     |  13 +-
 .../nifi/processors/influxdb/PutInfluxDB.java   |   5 +-
 .../jms/cf/JMSConnectionFactoryProvider.java    |  11 +-
 .../jms/processors/AbstractJMSProcessor.java    |   7 +-
 .../apache/nifi/jms/processors/ConsumeJMS.java  |   7 +-
 .../apache/nifi/jms/processors/PublishJMS.java  |   2 +-
 .../kafka/pubsub/ConsumeKafkaRecord_0_10.java   |   9 +-
 .../kafka/pubsub/ConsumeKafka_0_10.java         |   7 +-
 .../kafka/pubsub/KafkaProcessorUtils.java       |  11 +-
 .../kafka/pubsub/PublishKafkaRecord_0_10.java   |  15 +-
 .../kafka/pubsub/PublishKafka_0_10.java         |  13 +-
 .../kafka/pubsub/ConsumeKafkaRecord_0_11.java   |  13 +-
 .../kafka/pubsub/ConsumeKafka_0_11.java         |  11 +-
 .../kafka/pubsub/KafkaProcessorUtils.java       |  11 +-
 .../kafka/pubsub/PublishKafkaRecord_0_11.java   |  19 +-
 .../kafka/pubsub/PublishKafka_0_11.java         |  17 +-
 .../apache/nifi/processors/kafka/GetKafka.java  |  19 +-
 .../apache/nifi/processors/kafka/PutKafka.java  |  19 +-
 .../processors/kafka/pubsub/ConsumeKafka.java   |   7 +-
 .../kafka/pubsub/KafkaProcessorUtils.java       |   7 +-
 .../processors/kafka/pubsub/PublishKafka.java   |  13 +-
 .../kafka/pubsub/ConsumeKafkaRecord_1_0.java    |  13 +-
 .../kafka/pubsub/ConsumeKafka_1_0.java          |  11 +-
 .../kafka/pubsub/KafkaProcessorUtils.java       |  11 +-
 .../kafka/pubsub/PublishKafkaRecord_1_0.java    |  19 +-
 .../kafka/pubsub/PublishKafka_1_0.java          |  17 +-
 .../processors/kite/AbstractKiteProcessor.java  |   3 +-
 .../nifi/processors/kite/ConvertAvroSchema.java |   7 +-
 .../nifi/processors/kite/ConvertCSVToAvro.java  |  15 +-
 .../nifi/processors/kite/ConvertJSONToAvro.java |   5 +-
 .../nifi/processors/kite/InferAvroSchema.java   |  17 +-
 .../processors/kite/StoreInKiteDataset.java     |   3 +-
 .../nifi/processors/kudu/AbstractKudu.java      |   7 +-
 .../nifi/processors/yandex/YandexTranslate.java |  11 +-
 .../nifi/processors/image/ResizeImage.java      |   5 +-
 .../processors/media/ExtractMediaMetadata.java  |   3 +-
 .../service/GraphiteMetricReporterService.java  |   7 +-
 .../reporting/task/MetricsReportingTask.java    |   3 +-
 .../mongodb/AbstractMongoProcessor.java         |  14 +-
 .../nifi/processors/mongodb/GetMongo.java       |  17 +-
 .../processors/mongodb/RunMongoAggregation.java |  20 +-
 .../nifi/processors/mongodb/GetMongoIT.java     |   1 -
 .../mongodb/RunMongoAggregationIT.java          |   1 -
 .../AbstractMongoDBControllerService.java       |   7 +-
 .../nifi/processors/mqtt/ConsumeMQTT.java       |   3 +-
 .../nifi/processors/mqtt/PublishMQTT.java       |   7 +-
 .../nifi/processors/parquet/PutParquet.java     |   9 +-
 .../poi/ConvertExcelToCSVProcessor.java         |   3 +-
 .../org/apache/nifi/redis/util/RedisUtils.java  |   9 +-
 .../services/AvroSchemaRegistry.java            |   3 +-
 .../rethinkdb/AbstractRethinkDBProcessor.java   |   7 +-
 .../processors/rethinkdb/DeleteRethinkDB.java   |   3 +-
 .../nifi/processors/rethinkdb/GetRethinkDB.java |   3 +-
 .../nifi/processors/rethinkdb/PutRethinkDB.java |   3 +-
 .../nifi/processors/riemann/PutRiemann.java     |  22 +-
 .../lookup/script/ScriptedLookupService.java    |   3 +-
 .../nifi/processors/script/ExecuteScript.java   |   5 +-
 .../script/InvokeScriptedProcessor.java         |   6 +-
 .../reporting/script/ScriptedReportingTask.java |   5 +-
 .../AbstractScriptedControllerService.java      |   3 +-
 .../nifi/script/ScriptingComponentHelper.java   |   3 +-
 .../nifi/script/ScriptingComponentUtils.java    |   7 +-
 .../AbstractSiteToSiteReportingTask.java        |   7 +-
 .../SiteToSiteBulletinReportingTask.java        |   3 +-
 .../SiteToSiteProvenanceReportingTask.java      |   3 +-
 .../SiteToSiteStatusReportingTask.java          |   7 +-
 .../apache/nifi/processors/slack/PutSlack.java  |  15 +-
 .../apache/nifi/processors/solr/GetSolr.java    |   3 +-
 .../processors/solr/PutSolrContentStream.java   |  15 +-
 .../apache/nifi/processors/solr/SolrUtils.java  |   9 +-
 .../controller/livy/LivySessionController.java  |  17 +-
 .../livy/ExecuteSparkInteractive.java           |   7 +-
 .../AbstractDatabaseFetchProcessor.java         |  13 +-
 .../nifi/processors/standard/ControlRate.java   |   5 +-
 .../standard/ConvertCharacterSet.java           |   5 +-
 .../processors/standard/ConvertJSONToSQL.java   |  11 +-
 .../processors/standard/DetectDuplicate.java    |   5 +-
 .../nifi/processors/standard/EnforceOrder.java  |  15 +-
 .../processors/standard/EvaluateJsonPath.java   |  28 ++-
 .../nifi/processors/standard/EvaluateXPath.java |  13 +-
 .../processors/standard/EvaluateXQuery.java     |  10 +-
 .../processors/standard/ExecuteProcess.java     |  11 +-
 .../nifi/processors/standard/ExecuteSQL.java    |   7 +-
 .../standard/ExecuteStreamCommand.java          |  46 ++--
 .../nifi/processors/standard/ExtractText.java   |   3 +-
 .../standard/FetchDistributedMapCache.java      |   5 +-
 .../nifi/processors/standard/FetchFile.java     |   7 +-
 .../processors/standard/FetchFileTransfer.java  |  13 +-
 .../nifi/processors/standard/FlattenJson.java   |   5 +-
 .../processors/standard/GenerateFlowFile.java   |   8 +-
 .../processors/standard/GenerateTableFetch.java |   8 +-
 .../nifi/processors/standard/GetFile.java       |   3 +-
 .../nifi/processors/standard/GetHTTP.java       |  12 +-
 .../processors/standard/HandleHttpRequest.java  |   9 +-
 .../processors/standard/HandleHttpResponse.java |   9 +-
 .../nifi/processors/standard/InvokeHTTP.java    |  25 ++-
 .../processors/standard/JoltTransformJSON.java  |  11 +-
 .../nifi/processors/standard/ListFile.java      |   3 +-
 .../processors/standard/ListFileTransfer.java   |   9 +-
 .../nifi/processors/standard/ListenHTTP.java    |   5 +-
 .../processors/standard/ListenUDPRecord.java    |  11 +-
 .../nifi/processors/standard/LogMessage.java    |  19 +-
 .../processors/standard/LookupAttribute.java    |   5 +-
 .../nifi/processors/standard/LookupRecord.java  |  11 +-
 .../nifi/processors/standard/MergeContent.java  |  24 +-
 .../nifi/processors/standard/MergeRecord.java   |   3 +-
 .../nifi/processors/standard/ModifyBytes.java   |   5 +-
 .../processors/standard/MonitorActivity.java    |   5 +-
 .../apache/nifi/processors/standard/Notify.java |   9 +-
 .../processors/standard/PartitionRecord.java    |   5 +-
 .../nifi/processors/standard/PostHTTP.java      |   5 +-
 .../processors/standard/PutDatabaseRecord.java  |  15 +-
 .../standard/PutDistributedMapCache.java        |   5 +-
 .../nifi/processors/standard/PutEmail.java      |  33 +--
 .../nifi/processors/standard/PutFile.java       |  11 +-
 .../apache/nifi/processors/standard/PutSQL.java |   3 +-
 .../nifi/processors/standard/PutSyslog.java     |  11 +-
 .../processors/standard/QueryDatabaseTable.java |  13 +-
 .../nifi/processors/standard/QueryRecord.java   |  15 +-
 .../nifi/processors/standard/ReplaceText.java   |   5 +-
 .../standard/ReplaceTextWithMapping.java        |   5 +-
 .../processors/standard/RouteOnAttribute.java   |   8 +-
 .../processors/standard/RouteOnContent.java     |   8 +-
 .../nifi/processors/standard/RouteText.java     |   9 +-
 .../nifi/processors/standard/SplitRecord.java   |   3 +-
 .../nifi/processors/standard/TailFile.java      |   9 +-
 .../nifi/processors/standard/TransformXml.java  |   8 +-
 .../nifi/processors/standard/UpdateCounter.java |   5 +-
 .../nifi/processors/standard/UpdateRecord.java  |   7 +-
 .../nifi/processors/standard/ValidateCsv.java   |   9 +-
 .../apache/nifi/processors/standard/Wait.java   |  15 +-
 .../processors/standard/util/FTPTransfer.java   |   3 +-
 .../processors/standard/util/FileTransfer.java  |  19 +-
 .../processors/standard/util/JdbcCommon.java    |   5 +-
 .../processors/standard/util/JmsProperties.java |   5 +-
 .../processors/standard/util/SFTPTransfer.java  |   7 +-
 .../standard/TestExecuteStreamCommand.java      |  17 --
 .../nifi/processors/standard/TestFTP.java       |  21 +-
 .../nifi/processors/standard/TestGetFile.java   |   1 +
 .../standard/TestJoltTransformJSON.java         |  20 +-
 .../processors/standard/TestLogMessage.java     |  12 +-
 .../processors/standard/TestMergeContent.java   |  17 +-
 .../processors/standard/TestQueryRecord.java    |  25 ++-
 .../processors/standard/TestReplaceText.java    | 224 +++++++------------
 .../standard/TestReplaceTextWithMapping.java    |  46 ++--
 .../processors/standard/TestUpdateRecord.java   |   5 -
 .../apache/nifi/dbcp/DBCPConnectionPool.java    |  17 +-
 .../apache/nifi/hbase/HBaseClientService.java   |   3 +-
 .../HBase_1_1_2_ClientMapCacheService.java      |   7 +-
 .../nifi/http/StandardHttpContextMap.java       |   3 +-
 .../hortonworks/HortonworksSchemaRegistry.java  |   3 +-
 .../nifi/lookup/CSVRecordLookupService.java     |  11 +-
 .../nifi/lookup/SimpleCsvFileLookupService.java |  15 +-
 .../lookup/SimpleKeyValueLookupService.java     |   3 +-
 .../CommonsConfigurationLookupService.java      |   5 +-
 .../nifi/lookup/maxmind/IPLookupService.java    |  13 +-
 .../java/org/apache/nifi/csv/CSVReader.java     |   3 +-
 .../java/org/apache/nifi/grok/GrokReader.java   |   3 +-
 .../org/apache/nifi/json/JsonPathReader.java    |   3 +-
 .../apache/nifi/json/JsonRecordSetWriter.java   |   3 +-
 .../nifi/text/FreeFormTextRecordSetWriter.java  |   5 +-
 .../analysis/AttributeRollingWindow.java        |   3 +-
 .../apache/nifi/processors/gettcp/GetTCP.java   |  10 +-
 .../processors/attributes/UpdateAttribute.java  |   7 +-
 .../nifi/processors/websocket/PutWebSocket.java |   9 +-
 .../websocket/jetty/JettyWebSocketClient.java   |  13 +-
 .../websocket/jetty/JettyWebSocketServer.java   |   5 +-
 314 files changed, 2192 insertions(+), 1691 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperty.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperty.java b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperty.java
index e7712df..aa52226 100644
--- a/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperty.java
+++ b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperty.java
@@ -24,6 +24,7 @@ import java.lang.annotation.RetentionPolicy;
 import java.lang.annotation.Target;
 
 import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 
 /**
  * An annotation that may be placed on a {@link ConfigurableComponent} to
@@ -38,9 +39,13 @@ public @interface DynamicProperty {
 
     String name();
 
+    @Deprecated
     boolean supportsExpressionLanguage() default false;
 
     String value();
 
     String description();
+
+    ExpressionLanguageScope expressionLanguageScope() default ExpressionLanguageScope.NONE;
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-api/src/main/java/org/apache/nifi/components/PropertyDescriptor.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/components/PropertyDescriptor.java b/nifi-api/src/main/java/org/apache/nifi/components/PropertyDescriptor.java
index 1299d3d..3ac7510 100644
--- a/nifi-api/src/main/java/org/apache/nifi/components/PropertyDescriptor.java
+++ b/nifi-api/src/main/java/org/apache/nifi/components/PropertyDescriptor.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 
 /**
  * An immutable object for holding information about a type of component
@@ -78,8 +79,14 @@ public final class PropertyDescriptor implements Comparable<PropertyDescriptor>
      * indicates whether or not this property supports the Attribute Expression
      * Language
      */
+    @Deprecated
     private final boolean expressionLanguageSupported;
     /**
+     * indicates whether or nor this property will evaluate expression language
+     * against the flow file attributes
+     */
+    private final ExpressionLanguageScope expressionLanguageScope;
+    /**
      * indicates whether or not this property represents resources that should be added
      * to the classpath for this instance of the component
      */
@@ -109,6 +116,7 @@ public final class PropertyDescriptor implements Comparable<PropertyDescriptor>
         this.dynamic = builder.dynamic;
         this.dynamicallyModifiesClasspath = builder.dynamicallyModifiesClasspath;
         this.expressionLanguageSupported = builder.expressionLanguageSupported;
+        this.expressionLanguageScope = builder.expressionLanguageScope;
         this.controllerServiceDefinition = builder.controllerServiceDefinition;
         this.validators = new ArrayList<>(builder.validators);
     }
@@ -236,7 +244,11 @@ public final class PropertyDescriptor implements Comparable<PropertyDescriptor>
         private List<AllowableValue> allowableValues = null;
         private boolean required = false;
         private boolean sensitive = false;
+
+        @Deprecated
         private boolean expressionLanguageSupported = false;
+
+        private ExpressionLanguageScope expressionLanguageScope = ExpressionLanguageScope.NONE;
         private boolean dynamic = false;
         private boolean dynamicallyModifiesClasspath = false;
         private Class<? extends ControllerService> controllerServiceDefinition;
@@ -253,6 +265,7 @@ public final class PropertyDescriptor implements Comparable<PropertyDescriptor>
             this.dynamic = specDescriptor.dynamic;
             this.dynamicallyModifiesClasspath = specDescriptor.dynamicallyModifiesClasspath;
             this.expressionLanguageSupported = specDescriptor.expressionLanguageSupported;
+            this.expressionLanguageScope = specDescriptor.expressionLanguageScope;
             this.controllerServiceDefinition = specDescriptor.getControllerServiceDefinition();
             this.validators = new ArrayList<>(specDescriptor.validators);
             return this;
@@ -297,12 +310,24 @@ public final class PropertyDescriptor implements Comparable<PropertyDescriptor>
          * @param supported true if yes; false otherwise
          * @return the builder
          */
+        @Deprecated
         public Builder expressionLanguageSupported(final boolean supported) {
             this.expressionLanguageSupported = supported;
             return this;
         }
 
         /**
+         * Sets the scope of the expression language evaluation
+         *
+         * @param expressionLanguageScope scope of the expression language evaluation
+         * @return the builder
+         */
+        public Builder expressionLanguageSupported(final ExpressionLanguageScope expressionLanguageScope) {
+            this.expressionLanguageScope = expressionLanguageScope;
+            return this;
+        }
+
+        /**
          * @param description of the property
          * @return the builder
          */
@@ -513,7 +538,11 @@ public final class PropertyDescriptor implements Comparable<PropertyDescriptor>
     }
 
     public boolean isExpressionLanguageSupported() {
-        return expressionLanguageSupported;
+        return expressionLanguageSupported || !expressionLanguageScope.equals(ExpressionLanguageScope.NONE);
+    }
+
+    public ExpressionLanguageScope getExpressionLanguageScope() {
+        return expressionLanguageScope;
     }
 
     public boolean isDynamicClasspathModifier() {

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-api/src/main/java/org/apache/nifi/expression/ExpressionLanguageScope.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/expression/ExpressionLanguageScope.java b/nifi-api/src/main/java/org/apache/nifi/expression/ExpressionLanguageScope.java
new file mode 100644
index 0000000..4ad7f2c
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/expression/ExpressionLanguageScope.java
@@ -0,0 +1,68 @@
+/*
+ * 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.expression;
+
+/**
+ * Indicates the scope of expression language on a property descriptor.
+ *
+ * Scope of the expression language is hierarchical.
+ *      NONE -> VARIABLE_REGISTRY -> FLOWFILE_ATTRIBUTES
+ *
+ * When scope is set to FlowFiles attributes, variables are evaluated
+ * against attributes of each incoming flow file. If no matching attribute
+ * is found, variable registry will be checked.
+ *
+ * NONE - expression language is not supported
+ *
+ * VARIABLE_REGISTRY is hierarchically constructed as below:
+ *  |---- Variables defined at process group level and then, recursively, up
+ *  |     to the higher process group until the root process group.
+ *  |--- Variables defined in custom properties files through the
+ *  |    nifi.variable.registry.properties property in nifi.properties file.
+ *  |-- Environment variables defined at JVM level and system properties.
+ *
+ * FLOWFILE_ATTRIBUTES - will check attributes of each individual flow file
+ *
+ */
+public enum ExpressionLanguageScope {
+
+    /**
+     * Expression language is disabled
+     */
+    NONE("Not Supported"),
+
+    /**
+     * Expression language is evaluated against variables in registry
+     */
+    VARIABLE_REGISTRY("Variable Registry Only"),
+
+    /**
+     * Expression language is evaluated per flow file using attributes
+     */
+    FLOWFILE_ATTRIBUTES("Variable Registry and FlowFile Attributes");
+
+    private String description;
+
+    private ExpressionLanguageScope(String description) {
+        this.description = description;
+    }
+
+    public String getDescription() {
+        return this.description;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/email/EmailNotificationService.java
----------------------------------------------------------------------
diff --git a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/email/EmailNotificationService.java b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/email/EmailNotificationService.java
index 91451fe..588b069 100644
--- a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/email/EmailNotificationService.java
+++ b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/email/EmailNotificationService.java
@@ -42,6 +42,7 @@ import org.apache.nifi.bootstrap.notification.NotificationType;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 
@@ -58,20 +59,20 @@ public class EmailNotificationService extends AbstractNotificationService {
         .description("The Port used for SMTP communications")
         .required(true)
         .defaultValue("25")
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .addValidator(StandardValidators.PORT_VALIDATOR)
         .build();
     public static final PropertyDescriptor SMTP_USERNAME = new PropertyDescriptor.Builder()
         .name("SMTP Username")
         .description("Username for the SMTP account")
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .required(false)
         .build();
     public static final PropertyDescriptor SMTP_PASSWORD = new PropertyDescriptor.Builder()
         .name("SMTP Password")
         .description("Password for the SMTP account")
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .required(false)
         .sensitive(true)
@@ -80,7 +81,7 @@ public class EmailNotificationService extends AbstractNotificationService {
         .name("SMTP Auth")
         .description("Flag indicating whether authentication should be used")
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
         .defaultValue("true")
         .build();
@@ -88,7 +89,7 @@ public class EmailNotificationService extends AbstractNotificationService {
         .name("SMTP TLS")
         .description("Flag indicating whether TLS should be enabled")
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
         .defaultValue("false")
         .build();
@@ -96,7 +97,7 @@ public class EmailNotificationService extends AbstractNotificationService {
         .name("SMTP Socket Factory")
         .description("Socket Factory to use for SMTP Connection")
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .defaultValue("javax.net.ssl.SSLSocketFactory")
         .build();
@@ -104,7 +105,7 @@ public class EmailNotificationService extends AbstractNotificationService {
         .name("SMTP X-Mailer Header")
         .description("X-Mailer used in the header of the outgoing email")
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .defaultValue("NiFi")
         .build();
@@ -112,7 +113,7 @@ public class EmailNotificationService extends AbstractNotificationService {
         .name("Content Type")
         .description("Mime Type used to interpret the contents of the email, such as text/plain or text/html")
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .defaultValue("text/plain")
         .build();
@@ -120,28 +121,28 @@ public class EmailNotificationService extends AbstractNotificationService {
         .name("From")
         .description("Specifies the Email address to use as the sender")
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .build();
     public static final PropertyDescriptor TO = new PropertyDescriptor.Builder()
         .name("To")
         .description("The recipients to include in the To-Line of the email")
         .required(false)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .build();
     public static final PropertyDescriptor CC = new PropertyDescriptor.Builder()
         .name("CC")
         .description("The recipients to include in the CC-Line of the email")
         .required(false)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .build();
     public static final PropertyDescriptor BCC = new PropertyDescriptor.Builder()
         .name("BCC")
         .description("The recipients to include in the BCC-Line of the email")
         .required(false)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/http/HttpNotificationService.java
----------------------------------------------------------------------
diff --git a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/http/HttpNotificationService.java b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/http/HttpNotificationService.java
index 0e9a004..14d6709 100644
--- a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/http/HttpNotificationService.java
+++ b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/http/HttpNotificationService.java
@@ -30,6 +30,7 @@ import org.apache.nifi.bootstrap.notification.NotificationType;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.security.util.SslContextFactory;
@@ -54,21 +55,21 @@ public class HttpNotificationService extends AbstractNotificationService {
             .name("URL")
             .description("The URL to send the notification to.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.URL_VALIDATOR)
             .build();
 
     public static final PropertyDescriptor PROP_CONNECTION_TIMEOUT = new PropertyDescriptor.Builder()
             .name("Connection timeout")
             .description("Max wait time for connection to remote service.")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
             .defaultValue("10s")
             .build();
     public static final PropertyDescriptor PROP_WRITE_TIMEOUT = new PropertyDescriptor.Builder()
             .name("Write timeout")
             .description("Max wait time for remote service to read the request sent.")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
             .defaultValue("10s")
             .build();
@@ -163,7 +164,7 @@ public class HttpNotificationService extends AbstractNotificationService {
                 .name(propertyDescriptorName)
                 .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
                 .dynamic(true)
-                .expressionLanguageSupported(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
                 .build();
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-docs/src/main/asciidoc/expression-language-guide.adoc
----------------------------------------------------------------------
diff --git a/nifi-docs/src/main/asciidoc/expression-language-guide.adoc b/nifi-docs/src/main/asciidoc/expression-language-guide.adoc
index 079cb8a..56906d8 100644
--- a/nifi-docs/src/main/asciidoc/expression-language-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/expression-language-guide.adoc
@@ -147,8 +147,18 @@ icon (
 image:iconInfo.png["Info"]
 ) next to the name of the Property. Hovering over this icon with the mouse will provide a tooltip that
 provides helpful information about the Property. This information includes a description of the Property,
-the default value (if any), historically configured values (if any), and whether or not this Property
-supports the expression language.
+the default value (if any), historically configured values (if any), and the evaluation scope of this
+property for expression language. There are three values and the evaluation scope of the expression 
+language is hierarchical: NONE -> VARIABLE_REGISTRY -> FLOWFILE_ATTRIBUTES.
+
+* NONE - expression language is not supported for this property
+* VARIABLE_REGISTRY is hierarchically constructed as below:
+** Variables defined at process group level and then, recursively, up to the higher process group until 
+the root process group.
+** Variables defined in custom properties files through the nifi.variable.registry.properties property 
+in nifi.properties file.
+** Environment variables defined at JVM level and system properties.
+* FLOWFILE_ATTRIBUTES - will use attributes of each individual flow file.
 
 
 [[editor]]

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceLookup.java
----------------------------------------------------------------------
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceLookup.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceLookup.java
index e07540b..9893a96 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceLookup.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceLookup.java
@@ -21,6 +21,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.ControllerServiceLookup;
 
@@ -97,4 +98,8 @@ public abstract class MockControllerServiceLookup implements ControllerServiceLo
         final ControllerServiceConfiguration status = controllerServiceMap.get(serviceIdentifier);
         return status == null ? null : serviceIdentifier;
     }
+
+    public InputRequirement getInputRequirement() {
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
----------------------------------------------------------------------
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
index afd8456..3448524 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
@@ -28,6 +28,7 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
 
+import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.attribute.expression.language.Query;
 import org.apache.nifi.attribute.expression.language.Query.Range;
 import org.apache.nifi.components.ConfigurableComponent;
@@ -58,6 +59,7 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
     private boolean allowExpressionValidation = true;
     private volatile boolean incomingConnection = true;
     private volatile boolean nonLoopConnection = true;
+    private volatile InputRequirement inputRequirement = null;
     private int maxConcurrentTasks = 1;
 
     private volatile Set<Relationship> connections = new HashSet<>();
@@ -67,7 +69,7 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
     private volatile boolean isPrimaryNode;
 
     public MockProcessContext(final ConfigurableComponent component) {
-        this(component, new MockStateManager(component),VariableRegistry.EMPTY_REGISTRY);
+        this(component, new MockStateManager(component), VariableRegistry.EMPTY_REGISTRY);
     }
 
     /**
@@ -79,6 +81,7 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
      */
     public MockProcessContext(final ConfigurableComponent component, final StateManager stateManager, final VariableRegistry variableRegistry) {
         this.component = Objects.requireNonNull(component);
+        this.inputRequirement = component.getClass().getAnnotation(InputRequirement.class);
         this.stateManager = stateManager;
         this.variableRegistry = variableRegistry;
     }
@@ -410,4 +413,10 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
         }
         isPrimaryNode = primaryNode;
     }
+
+    @Override
+    public InputRequirement getInputRequirement() {
+        return inputRequirement;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-mock/src/main/java/org/apache/nifi/util/MockPropertyValue.java
----------------------------------------------------------------------
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 08c8e97..9b825d9 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
@@ -20,14 +20,16 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.attribute.expression.language.Query;
-import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
 import org.apache.nifi.attribute.expression.language.Query.Range;
+import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
 import org.apache.nifi.components.PropertyDescriptor;
 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.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.exception.ProcessException;
@@ -36,10 +38,12 @@ import org.apache.nifi.registry.VariableRegistry;
 public class MockPropertyValue implements PropertyValue {
     private final String rawValue;
     private final Boolean expectExpressions;
-    private final ControllerServiceLookup serviceLookup;
+    private final ExpressionLanguageScope expressionLanguageScope;
+    private final MockControllerServiceLookup serviceLookup;
     private final PropertyDescriptor propertyDescriptor;
     private final PropertyValue stdPropValue;
     private final VariableRegistry variableRegistry;
+
     private boolean expressionsEvaluated = false;
 
     public MockPropertyValue(final String rawValue) {
@@ -59,12 +63,12 @@ public class MockPropertyValue implements PropertyValue {
     }
 
     private MockPropertyValue(final String rawValue, final ControllerServiceLookup serviceLookup, final PropertyDescriptor propertyDescriptor, final boolean alreadyEvaluated,
-                              final VariableRegistry variableRegistry) {
+            final VariableRegistry variableRegistry) {
         this.stdPropValue = new StandardPropertyValue(rawValue, serviceLookup, variableRegistry);
-
         this.rawValue = rawValue;
-        this.serviceLookup = serviceLookup;
+        this.serviceLookup = (MockControllerServiceLookup) serviceLookup;
         this.expectExpressions = propertyDescriptor == null ? null : propertyDescriptor.isExpressionLanguageSupported();
+        this.expressionLanguageScope = propertyDescriptor == null ? null : propertyDescriptor.getExpressionLanguageScope();
         this.propertyDescriptor = propertyDescriptor;
         this.expressionsEvaluated = alreadyEvaluated;
         this.variableRegistry = variableRegistry;
@@ -80,6 +84,55 @@ public class MockPropertyValue implements PropertyValue {
         }
     }
 
+    private void validateExpressionScope(boolean attributesAvailable) {
+        // language scope is not null, we have attributes available but scope is not equal to FF attributes
+        // it means that we're not evaluating against flow file attributes even though attributes are available
+        if(expressionLanguageScope != null
+                && (attributesAvailable && !ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope))) {
+            throw new IllegalStateException("Attempting to evaluate expression language for " + propertyDescriptor.getName()
+                    + " using flow file attributes but the scope evaluation is set to " + expressionLanguageScope + ". The"
+                    + " proper scope should be set in the property descriptor using"
+                    + " PropertyDescriptor.Builder.expressionLanguageSupported(ExpressionLanguageScope)");
+        }
+
+        // if the service lookup is an instance of the validation context, we're in the validate() method
+        // at this point we don't have any flow file available and we should not care about the scope
+        // even though it is defined as FLOWFILE_ATTRIBUTES
+        if(expressionLanguageScope != null
+                && ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope)
+                && this.serviceLookup instanceof MockValidationContext) {
+            return;
+        }
+
+        // we check if the input requirement is INPUT_FORBIDDEN
+        // in that case, we don't care if attributes are not available even though scope is FLOWFILE_ATTRIBUTES
+        // it likely means that the property has been defined in a common/abstract class used by multiple processors with
+        // different input requirements.
+        if(expressionLanguageScope != null
+                && ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope)
+                && (this.serviceLookup.getInputRequirement() == null
+                    || this.serviceLookup.getInputRequirement().value().equals(InputRequirement.Requirement.INPUT_FORBIDDEN))) {
+            return;
+        }
+
+        // if we have a processor where input requirement is INPUT_ALLOWED, we need to check if there is an
+        // incoming connection or not. If not, we don't care if attributes are not available even though scope is FLOWFILE_ATTRIBUTES
+        if(expressionLanguageScope != null
+                && ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope)
+                && !((MockProcessContext) this.serviceLookup).hasIncomingConnection()) {
+            return;
+        }
+
+        // we're trying to evaluate against flow files attributes but we don't have any attributes available.
+        if(expressionLanguageScope != null
+                && (!attributesAvailable && ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope))) {
+            throw new IllegalStateException("Attempting to evaluate expression language for " + propertyDescriptor.getName()
+                    + " without using flow file attributes but the scope evaluation is set to " + expressionLanguageScope + ". The"
+                    + " proper scope should be set in the property descriptor using"
+                    + " PropertyDescriptor.Builder.expressionLanguageSupported(ExpressionLanguageScope)");
+        }
+    }
+
     @Override
     public String getValue() {
         ensureExpressionsEvaluated();
@@ -185,6 +238,8 @@ public class MockPropertyValue implements PropertyValue {
             return this;
         }
 
+        validateExpressionScope(flowFile != null || additionalAttributes != null);
+
         final PropertyValue newValue = stdPropValue.evaluateAttributeExpressions(flowFile, additionalAttributes, decorator, stateValues);
         return new MockPropertyValue(newValue.getValue(), serviceLookup, propertyDescriptor, true, variableRegistry);
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java
----------------------------------------------------------------------
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 564ec54..72820ee 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
@@ -35,7 +35,7 @@ import org.apache.nifi.expression.ExpressionLanguageCompiler;
 import org.apache.nifi.registry.VariableRegistry;
 
 
-public class MockValidationContext implements ValidationContext, ControllerServiceLookup {
+public class MockValidationContext extends MockControllerServiceLookup implements ValidationContext, ControllerServiceLookup {
 
     private final MockProcessContext context;
     private final Map<String, Boolean> expressionLanguageSupported;

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-reporting-task/src/main/java/org/apache/nifi/reporting/ambari/AmbariReportingTask.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-reporting-task/src/main/java/org/apache/nifi/reporting/ambari/AmbariReportingTask.java b/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-reporting-task/src/main/java/org/apache/nifi/reporting/ambari/AmbariReportingTask.java
index a5ce9f4..5bbdecb 100644
--- a/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-reporting-task/src/main/java/org/apache/nifi/reporting/ambari/AmbariReportingTask.java
+++ b/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-reporting-task/src/main/java/org/apache/nifi/reporting/ambari/AmbariReportingTask.java
@@ -25,6 +25,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.status.ProcessGroupStatus;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.reporting.AbstractReportingTask;
 import org.apache.nifi.reporting.ReportingContext;
@@ -60,7 +61,7 @@ public class AmbariReportingTask extends AbstractReportingTask {
             .name("Metrics Collector URL")
             .description("The URL of the Ambari Metrics Collector Service")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue("http://localhost:6188/ws/v1/timeline/metrics")
             .addValidator(StandardValidators.URL_VALIDATOR)
             .build();
@@ -69,7 +70,7 @@ public class AmbariReportingTask extends AbstractReportingTask {
             .name("Application ID")
             .description("The Application ID to be included in the metrics sent to Ambari")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue("nifi")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
@@ -78,7 +79,7 @@ public class AmbariReportingTask extends AbstractReportingTask {
             .name("Hostname")
             .description("The Hostname of this NiFi instance to be included in the metrics sent to Ambari")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue("${hostname(true)}")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
@@ -88,7 +89,7 @@ public class AmbariReportingTask extends AbstractReportingTask {
             .description("If specified, the reporting task will send metrics about this process group only. If"
                     + " not, the root process group is used and global metrics are sent.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/main/java/org/apache/nifi/amqp/processors/PublishAMQP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/main/java/org/apache/nifi/amqp/processors/PublishAMQP.java b/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/main/java/org/apache/nifi/amqp/processors/PublishAMQP.java
index 7dce05e..4dbb7ed 100644
--- a/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/main/java/org/apache/nifi/amqp/processors/PublishAMQP.java
+++ b/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/main/java/org/apache/nifi/amqp/processors/PublishAMQP.java
@@ -38,6 +38,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -84,7 +85,7 @@ public class PublishAMQP extends AbstractAMQPProcessor<AMQPPublisher> {
                     + "It is an optional property. If kept empty the messages will be sent to a default AMQP exchange.")
             .required(true)
             .defaultValue("")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(Validator.VALID)
             .build();
     public static final PropertyDescriptor ROUTING_KEY = new PropertyDescriptor.Builder()
@@ -94,7 +95,7 @@ public class PublishAMQP extends AbstractAMQPProcessor<AMQPPublisher> {
                     + "corresponds to a destination queue name, otherwise a binding from the Exchange to a Queue via Routing Key must be set "
                     + "(usually by the AMQP administrator)")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/reporting/ReportLineageToAtlas.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/reporting/ReportLineageToAtlas.java b/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/reporting/ReportLineageToAtlas.java
index 4cfbce4..07d806a 100644
--- a/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/reporting/ReportLineageToAtlas.java
+++ b/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/reporting/ReportLineageToAtlas.java
@@ -16,7 +16,33 @@
  */
 package org.apache.nifi.atlas.reporting;
 
-import com.sun.jersey.api.client.ClientResponse;
+import static org.apache.commons.lang3.StringUtils.isEmpty;
+import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_BATCH_SIZE;
+import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_START_POSITION;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.time.Instant;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.ServiceLoader;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.stream.Stream;
+
 import org.apache.atlas.ApplicationProperties;
 import org.apache.atlas.AtlasServiceException;
 import org.apache.commons.lang3.StringUtils;
@@ -30,8 +56,8 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
-import org.apache.nifi.atlas.NiFiAtlasHook;
 import org.apache.nifi.atlas.NiFiAtlasClient;
+import org.apache.nifi.atlas.NiFiAtlasHook;
 import org.apache.nifi.atlas.NiFiFlow;
 import org.apache.nifi.atlas.NiFiFlowAnalyzer;
 import org.apache.nifi.atlas.provenance.AnalysisContext;
@@ -55,6 +81,7 @@ import org.apache.nifi.context.PropertyContext;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.status.ProcessGroupStatus;
 import org.apache.nifi.kerberos.KerberosCredentialsService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.provenance.ProvenanceEventRecord;
@@ -65,34 +92,7 @@ import org.apache.nifi.reporting.ReportingContext;
 import org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer;
 import org.apache.nifi.ssl.SSLContextService;
 
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.time.Instant;
-import java.time.ZoneOffset;
-import java.time.format.DateTimeFormatter;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.ServiceLoader;
-import java.util.Set;
-import java.util.function.Consumer;
-import java.util.stream.Stream;
-
-import static org.apache.commons.lang3.StringUtils.isEmpty;
-import static org.apache.nifi.atlas.reporting.ReportLineageToAtlas.NIFI_KERBEROS_KEYTAB;
-import static org.apache.nifi.atlas.reporting.ReportLineageToAtlas.NIFI_KERBEROS_PRINCIPAL;
-import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_BATCH_SIZE;
-import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_START_POSITION;
+import com.sun.jersey.api.client.ClientResponse;
 
 @Tags({"atlas", "lineage"})
 @CapabilityDescription("Report NiFi flow data set level lineage to Apache Atlas." +
@@ -102,7 +102,8 @@ import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.
         " in addition to NiFi provenance events providing detailed event level lineage." +
         " See 'Additional Details' for further description and limitations.")
 @Stateful(scopes = Scope.LOCAL, description = "Stores the Reporting Task's last event Id so that on restart the task knows where it left off.")
-@DynamicProperty(name = "hostnamePattern.<ClusterName>", value = "hostname Regex patterns", description = RegexClusterResolver.PATTERN_PROPERTY_PREFIX_DESC)
+@DynamicProperty(name = "hostnamePattern.<ClusterName>", value = "hostname Regex patterns",
+                 description = RegexClusterResolver.PATTERN_PROPERTY_PREFIX_DESC, expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY)
 // In order for each reporting task instance to have its own static objects such as KafkaNotification.
 @RequiresInstanceClassLoading
 public class ReportLineageToAtlas extends AbstractReportingTask {
@@ -115,7 +116,7 @@ public class ReportLineageToAtlas extends AbstractReportingTask {
                     " For accessing Atlas behind Knox gateway, specify Knox gateway URL" +
                     " (e.g. https://knox-hostname:8443/gateway/{topology-name}/atlas).")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
             .build();
 
@@ -135,7 +136,7 @@ public class ReportLineageToAtlas extends AbstractReportingTask {
             .displayName("Atlas Username")
             .description("User name to communicate with Atlas.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
             .build();
 
@@ -145,7 +146,7 @@ public class ReportLineageToAtlas extends AbstractReportingTask {
             .description("Password to communicate with Atlas.")
             .required(false)
             .sensitive(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
             .build();
 
@@ -156,7 +157,7 @@ public class ReportLineageToAtlas extends AbstractReportingTask {
                     " If not specified and 'Create Atlas Configuration File' is disabled," +
                     " then, 'atlas-application.properties' file under root classpath is used.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
             .build();
 
@@ -166,7 +167,7 @@ public class ReportLineageToAtlas extends AbstractReportingTask {
             .description("NiFi URL is used in Atlas to represent this NiFi cluster (or standalone instance)." +
                     " It is recommended to use one that can be accessible remotely instead of using 'localhost'.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.URL_VALIDATOR)
             .build();
 
@@ -178,7 +179,7 @@ public class ReportLineageToAtlas extends AbstractReportingTask {
                     " Cluster name mappings can be configured by user defined properties." +
                     " See additional detail for detail.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
             .build();
 
@@ -189,7 +190,7 @@ public class ReportLineageToAtlas extends AbstractReportingTask {
                     " automatically when this Reporting Task starts." +
                     " Note that the existing configuration file will be overwritten.")
             .required(true)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .allowableValues("true", "false")
             .defaultValue("false")
             .build();
@@ -210,7 +211,7 @@ public class ReportLineageToAtlas extends AbstractReportingTask {
                     " NOTE: Once this reporting task has started, restarting NiFi is required to changed this property" +
                     " as Atlas library holds a unmodifiable static reference to Kafka client.")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
             .build();
 
@@ -224,7 +225,7 @@ public class ReportLineageToAtlas extends AbstractReportingTask {
             .description("Protocol used to communicate with Kafka brokers to send Atlas hook notification messages." +
                     " Corresponds to Kafka's 'security.protocol' property.")
             .required(true)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .allowableValues(SEC_PLAINTEXT, SEC_SSL, SEC_SASL_PLAINTEXT, SEC_SASL_SSL)
             .defaultValue(SEC_PLAINTEXT.getValue())
             .build();
@@ -237,7 +238,7 @@ public class ReportLineageToAtlas extends AbstractReportingTask {
                     " This principal will be set into 'sasl.jaas.config' Kafka's property.")
             .required(false)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
     public static final PropertyDescriptor NIFI_KERBEROS_KEYTAB = new PropertyDescriptor.Builder()
             .name("nifi-kerberos-keytab")
@@ -247,7 +248,7 @@ public class ReportLineageToAtlas extends AbstractReportingTask {
                     " This principal will be set into 'sasl.jaas.config' Kafka's property.")
             .required(false)
             .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
     public static final PropertyDescriptor KERBEROS_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
         .name("kerberos-credentials-service")
@@ -267,7 +268,7 @@ public class ReportLineageToAtlas extends AbstractReportingTask {
                     " It is ignored unless one of the SASL options of the <Security Protocol> are selected.")
             .required(false)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue("kafka")
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/resolver/RegexClusterResolver.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/resolver/RegexClusterResolver.java b/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/resolver/RegexClusterResolver.java
index 8e0f5e0..d513c3a 100644
--- a/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/resolver/RegexClusterResolver.java
+++ b/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/resolver/RegexClusterResolver.java
@@ -20,6 +20,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.util.StandardValidators;
 
 import java.util.ArrayList;
@@ -47,7 +48,7 @@ public class RegexClusterResolver implements ClusterResolver {
                     .Builder().name(propertyDescriptorName)
                     .description(PATTERN_PROPERTY_PREFIX_DESC)
                     .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-                    .expressionLanguageSupported(true)
+                    .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
                     .dynamic(true)
                     .sensitive(false)
                     .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AbstractAWSProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AbstractAWSProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AbstractAWSProcessor.java
index 2e271cf..d066606 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AbstractAWSProcessor.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AbstractAWSProcessor.java
@@ -182,12 +182,14 @@ public abstract class AbstractAWSProcessor<ClientType extends AmazonWebServiceCl
         config.setConnectionTimeout(commsTimeout);
         config.setSocketTimeout(commsTimeout);
 
-        final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
-        if (sslContextService != null) {
-            final SSLContext sslContext = sslContextService.createSSLContext(SSLContextService.ClientAuth.NONE);
-            // NIFI-3788: Changed hostnameVerifier from null to DHV (BrowserCompatibleHostnameVerifier is deprecated)
-            SdkTLSSocketFactory sdkTLSSocketFactory = new SdkTLSSocketFactory(sslContext, new DefaultHostnameVerifier());
-            config.getApacheHttpClientConfig().setSslSocketFactory(sdkTLSSocketFactory);
+        if(this.getSupportedPropertyDescriptors().contains(SSL_CONTEXT_SERVICE)) {
+            final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
+            if (sslContextService != null) {
+                final SSLContext sslContext = sslContextService.createSSLContext(SSLContextService.ClientAuth.NONE);
+                // NIFI-3788: Changed hostnameVerifier from null to DHV (BrowserCompatibleHostnameVerifier is deprecated)
+                SdkTLSSocketFactory sdkTLSSocketFactory = new SdkTLSSocketFactory(sslContext, new DefaultHostnameVerifier());
+                config.getApacheHttpClientConfig().setSslSocketFactory(sdkTLSSocketFactory);
+            }
         }
 
         if (context.getProperty(PROXY_HOST).isSet()) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/CredentialPropertyDescriptors.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/CredentialPropertyDescriptors.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/CredentialPropertyDescriptors.java
index 920ec32..2a7296b 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/CredentialPropertyDescriptors.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/CredentialPropertyDescriptors.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.processors.aws.credentials.provider.factory;
 
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.util.StandardValidators;
 
 /**
@@ -37,7 +38,7 @@ public class CredentialPropertyDescriptors {
     public static final PropertyDescriptor USE_DEFAULT_CREDENTIALS = new PropertyDescriptor.Builder()
             .name("default-credentials")
             .displayName("Use Default Credentials")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(false)
             .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
             .sensitive(false)
@@ -50,7 +51,7 @@ public class CredentialPropertyDescriptors {
     public static final PropertyDescriptor CREDENTIALS_FILE = new PropertyDescriptor.Builder()
             .name("Credentials File")
             .displayName("Credentials File")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(false)
             .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
             .description("Path to a file containing AWS access key and secret key in properties file format.")
@@ -59,7 +60,7 @@ public class CredentialPropertyDescriptors {
     public static final PropertyDescriptor ACCESS_KEY = new PropertyDescriptor.Builder()
             .name("Access Key")
             .displayName("Access Key")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .sensitive(true)
@@ -68,7 +69,7 @@ public class CredentialPropertyDescriptors {
     public static final PropertyDescriptor SECRET_KEY = new PropertyDescriptor.Builder()
             .name("Secret Key")
             .displayName("Secret Key")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .sensitive(true)
@@ -83,7 +84,7 @@ public class CredentialPropertyDescriptors {
     public static final PropertyDescriptor PROFILE_NAME = new PropertyDescriptor.Builder()
             .name("profile-name")
             .displayName("Profile Name")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .sensitive(false)
@@ -93,7 +94,7 @@ public class CredentialPropertyDescriptors {
     public static final PropertyDescriptor USE_ANONYMOUS_CREDENTIALS = new PropertyDescriptor.Builder()
             .name("anonymous-credentials")
             .displayName("Use Anonymous Credentials")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(false)
             .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
             .sensitive(false)
@@ -110,7 +111,7 @@ public class CredentialPropertyDescriptors {
     public static final PropertyDescriptor ASSUME_ROLE_ARN = new PropertyDescriptor.Builder()
             .name("Assume Role ARN")
             .displayName("Assume Role ARN")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .sensitive(false)
@@ -123,7 +124,7 @@ public class CredentialPropertyDescriptors {
     public static final PropertyDescriptor ASSUME_ROLE_NAME = new PropertyDescriptor.Builder()
             .name("Assume Role Session Name")
             .displayName("Assume Role Session Name")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .sensitive(false)
@@ -148,7 +149,7 @@ public class CredentialPropertyDescriptors {
     public static final PropertyDescriptor ASSUME_ROLE_EXTERNAL_ID = new PropertyDescriptor.Builder()
             .name("assume-role-external-id")
             .displayName("Assume Role External ID")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .sensitive(false)
@@ -162,7 +163,7 @@ public class CredentialPropertyDescriptors {
     public static final PropertyDescriptor ASSUME_ROLE_PROXY_HOST = new PropertyDescriptor.Builder()
             .name("assume-role-proxy-host")
             .displayName("Assume Role Proxy Host")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .sensitive(false)
@@ -172,7 +173,7 @@ public class CredentialPropertyDescriptors {
     public static final PropertyDescriptor ASSUME_ROLE_PROXY_PORT = new PropertyDescriptor.Builder()
             .name("assume-role-proxy-port")
             .displayName("Assume Role Proxy Port")
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .required(false)
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
             .sensitive(false)

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/AbstractDynamoDBProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/AbstractDynamoDBProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/AbstractDynamoDBProcessor.java
index aed957f..6c5d4f6 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/AbstractDynamoDBProcessor.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/AbstractDynamoDBProcessor.java
@@ -31,6 +31,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -82,7 +83,7 @@ public abstract class AbstractDynamoDBProcessor extends AbstractAWSCredentialsPr
     public static final PropertyDescriptor TABLE = new PropertyDescriptor.Builder()
             .name("Table Name")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .description("The DynamoDB table name")
             .build();
@@ -90,7 +91,7 @@ public abstract class AbstractDynamoDBProcessor extends AbstractAWSCredentialsPr
     public static final PropertyDescriptor HASH_KEY_VALUE = new PropertyDescriptor.Builder()
             .name("Hash Key Value")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .description("The hash key value of the item")
             .defaultValue("${dynamodb.item.hash.key.value}")
@@ -100,7 +101,7 @@ public abstract class AbstractDynamoDBProcessor extends AbstractAWSCredentialsPr
             .name("Range Key Value")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("${dynamodb.item.range.key.value}")
             .build();
 
@@ -123,7 +124,7 @@ public abstract class AbstractDynamoDBProcessor extends AbstractAWSCredentialsPr
     public static final PropertyDescriptor HASH_KEY_NAME = new PropertyDescriptor.Builder()
             .name("Hash Key Name")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .description("The hash key name of the item")
             .build();
@@ -131,7 +132,7 @@ public abstract class AbstractDynamoDBProcessor extends AbstractAWSCredentialsPr
     public static final PropertyDescriptor RANGE_KEY_NAME = new PropertyDescriptor.Builder()
             .name("Range Key Name")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .description("The range key name of the item")
             .build();
@@ -139,7 +140,7 @@ public abstract class AbstractDynamoDBProcessor extends AbstractAWSCredentialsPr
     public static final PropertyDescriptor JSON_DOCUMENT = new PropertyDescriptor.Builder()
             .name("Json Document attribute")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .description("The Json document to be retrieved from the dynamodb item")
             .build();
@@ -147,7 +148,7 @@ public abstract class AbstractDynamoDBProcessor extends AbstractAWSCredentialsPr
     public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
             .name("Batch items for each request (between 1 and 50)")
             .required(false)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.createLongValidator(1, 50, true))
             .defaultValue("1")
             .description("The items to be retrieved in one batch")
@@ -158,7 +159,7 @@ public abstract class AbstractDynamoDBProcessor extends AbstractAWSCredentialsPr
             .description("Character set of data in the document")
             .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .defaultValue(Charset.defaultCharset().name())
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/firehose/AbstractKinesisFirehoseProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/firehose/AbstractKinesisFirehoseProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/firehose/AbstractKinesisFirehoseProcessor.java
index bf91e9b..215043d 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/firehose/AbstractKinesisFirehoseProcessor.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/firehose/AbstractKinesisFirehoseProcessor.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.processors.aws.kinesis.firehose;
 
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processors.aws.kinesis.AbstractBaseKinesisProcessor;
@@ -34,7 +35,7 @@ public abstract class AbstractKinesisFirehoseProcessor extends AbstractBaseKines
     public static final PropertyDescriptor KINESIS_FIREHOSE_DELIVERY_STREAM_NAME = new PropertyDescriptor.Builder()
             .name("Amazon Kinesis Firehose Delivery Stream Name")
             .description("The name of kinesis firehose delivery stream")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
@@ -72,6 +73,7 @@ public abstract class AbstractKinesisFirehoseProcessor extends AbstractBaseKines
      *
      * @deprecated use {@link #createClient(ProcessContext, AWSCredentialsProvider, ClientConfiguration)} instead
      */
+    @Deprecated
     @Override
     protected AmazonKinesisFirehoseClient createClient(final ProcessContext context, final AWSCredentials credentials, final ClientConfiguration config) {
         getLogger().info("Creating client using aws credentials");


[03/15] nifi git commit: NIFI-4149 - Indicate if EL is evaluated against FFs or not - take into account input requirement for documentation rendering - Renamed variable registry scope and added comments - Doc + change in mock framework to check scope + u

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UpdateCounter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UpdateCounter.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UpdateCounter.java
index 291736a..5115817 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UpdateCounter.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UpdateCounter.java
@@ -21,6 +21,7 @@ import org.apache.nifi.annotation.behavior.ReadsAttribute;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -51,7 +52,7 @@ public class UpdateCounter extends AbstractProcessor {
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .addValidator(StandardValidators.ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     static final PropertyDescriptor DELTA = new PropertyDescriptor.Builder()
@@ -62,7 +63,7 @@ public class UpdateCounter extends AbstractProcessor {
             .defaultValue("1")
             .addValidator(StandardValidators.INTEGER_VALIDATOR)
             .addValidator(StandardValidators.ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     static final Relationship SUCCESS = new Relationship.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UpdateRecord.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UpdateRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UpdateRecord.java
index 63e05ab..9b5d209 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UpdateRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UpdateRecord.java
@@ -40,6 +40,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.record.path.FieldValue;
@@ -87,7 +88,7 @@ public class UpdateRecord extends AbstractRecordProcessor {
         .description("Specifies how to interpret the configured replacement values")
         .allowableValues(LITERAL_VALUES, RECORD_PATH_VALUES)
         .defaultValue(LITERAL_VALUES.getValue())
-        .expressionLanguageSupported(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .required(true)
         .build();
 
@@ -105,7 +106,7 @@ public class UpdateRecord extends AbstractRecordProcessor {
             .description("Specifies the value to use to replace fields in the record that match the RecordPath: " + propertyDescriptorName)
             .required(false)
             .dynamic(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(new RecordPathPropertyNameValidator())
             .build();
     }
@@ -179,7 +180,7 @@ public class UpdateRecord extends AbstractRecordProcessor {
                         fieldVal.updateValue(evaluatedReplacementVal);
                     });
                 } else {
-                    final String evaluatedReplacementVal = replacementValue.getValue();
+                    final String evaluatedReplacementVal = replacementValue.evaluateAttributeExpressions(flowFile).getValue();
                     result.getSelectedFields().forEach(fieldVal -> fieldVal.updateValue(evaluatedReplacementVal));
                 }
             }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateCsv.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateCsv.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateCsv.java
index b661245..667d4f9 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateCsv.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateCsv.java
@@ -44,6 +44,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -120,7 +121,7 @@ public class ValidateCsv extends AbstractProcessor {
                     + "processors to apply. The following cell processors are allowed in the schema definition: "
                     + allowedOperators.toString() + ". Note: cell processors cannot be nested except with Optional.")
             .required(true)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
             .build();
 
@@ -140,7 +141,7 @@ public class ValidateCsv extends AbstractProcessor {
             .description("Character used as 'quote' in the incoming data. Example: \"")
             .required(true)
             .defaultValue("\"")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -150,7 +151,7 @@ public class ValidateCsv extends AbstractProcessor {
             .description("Character used as 'delimiter' in the incoming data. Example: ,")
             .required(true)
             .defaultValue(",")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
@@ -160,7 +161,7 @@ public class ValidateCsv extends AbstractProcessor {
             .description("Symbols used as 'end of line' in the incoming data. Example: \\n")
             .required(true)
             .defaultValue("\\n")
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Wait.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Wait.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Wait.java
index 928d179..cfec15e 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Wait.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Wait.java
@@ -48,6 +48,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.distributed.cache.client.AtomicDistributedMapCacheClient;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -110,7 +111,7 @@ public class Wait extends AbstractProcessor {
                 "be evaluated against a FlowFile in order to determine the release signal cache key")
             .required(true)
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(ResultType.STRING, true))
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor TARGET_SIGNAL_COUNT = new PropertyDescriptor.Builder()
@@ -123,7 +124,7 @@ public class Wait extends AbstractProcessor {
                     "otherwise checks against total count in a signal.")
             .required(true)
             .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("1")
             .build();
 
@@ -135,7 +136,7 @@ public class Wait extends AbstractProcessor {
                     "If not specified, this processor checks the total count in a signal.")
             .required(false)
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(ResultType.STRING, true))
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor WAIT_BUFFER_COUNT = new PropertyDescriptor.Builder()
@@ -159,7 +160,7 @@ public class Wait extends AbstractProcessor {
                     "Zero (0) has a special meaning, any number of FlowFiles can be released as long as signal count matches target.")
             .required(true)
             .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("1")
             .build();
 
@@ -171,7 +172,7 @@ public class Wait extends AbstractProcessor {
             .required(true)
             .defaultValue("10 min")
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
 
     public static final AllowableValue ATTRIBUTE_COPY_REPLACE = new AllowableValue("replace", "Replace if present",
@@ -187,7 +188,7 @@ public class Wait extends AbstractProcessor {
             .defaultValue(ATTRIBUTE_COPY_KEEP_ORIGINAL.getValue())
             .required(true)
             .allowableValues(ATTRIBUTE_COPY_REPLACE, ATTRIBUTE_COPY_KEEP_ORIGINAL)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
 
     public static final AllowableValue WAIT_MODE_TRANSFER_TO_WAIT = new AllowableValue("wait", "Transfer to wait relationship",
@@ -207,7 +208,7 @@ public class Wait extends AbstractProcessor {
             .defaultValue(WAIT_MODE_TRANSFER_TO_WAIT.getValue())
             .required(true)
             .allowableValues(WAIT_MODE_TRANSFER_TO_WAIT, WAIT_MODE_KEEP_IN_UPSTREAM)
-            .expressionLanguageSupported(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
             .build();
 
     public static final Relationship REL_SUCCESS = new Relationship.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FTPTransfer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FTPTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FTPTransfer.java
index 7a40612..91a5ac2 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FTPTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FTPTransfer.java
@@ -39,6 +39,7 @@ import org.apache.commons.net.ftp.FTPFile;
 import org.apache.commons.net.ftp.FTPHTTPClient;
 import org.apache.commons.net.ftp.FTPReply;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.DataUnit;
@@ -75,7 +76,7 @@ public class FTPTransfer implements FileTransfer {
         .addValidator(StandardValidators.PORT_VALIDATOR)
         .required(true)
         .defaultValue("21")
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .build();
     public static final PropertyDescriptor PROXY_TYPE = new PropertyDescriptor.Builder()
         .name("Proxy Type")

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileTransfer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileTransfer.java
index ac7f728..1163ea8 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileTransfer.java
@@ -24,6 +24,7 @@ import java.util.List;
 
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.util.StandardValidators;
 
@@ -62,14 +63,14 @@ public interface FileTransfer extends Closeable {
         .description("The fully qualified hostname or IP address of the remote system")
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .build();
     public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
         .name("Username")
         .description("Username")
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .required(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .build();
     public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
         .name("Password")
@@ -77,7 +78,7 @@ public interface FileTransfer extends Closeable {
         .addValidator(Validator.VALID)
         .required(false)
         .sensitive(true)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .build();
     public static final PropertyDescriptor DATA_TIMEOUT = new PropertyDescriptor.Builder()
         .name("Data Timeout")
@@ -98,7 +99,7 @@ public interface FileTransfer extends Closeable {
         .description("The path on the remote system from which to pull or push files")
         .required(false)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .build();
     public static final PropertyDescriptor CREATE_DIRECTORY = new PropertyDescriptor.Builder()
         .name("Create Directory")
@@ -217,7 +218,7 @@ public interface FileTransfer extends Closeable {
         .description("If set, the filename of the sent file will be equal to the value specified during the transfer and after successful "
             + "completion will be renamed to the original filename. If this value is set, the Dot Rename property is ignored.")
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .required(false)
         .build();
     public static final PropertyDescriptor LAST_MODIFIED_TIME = new PropertyDescriptor.Builder()
@@ -227,7 +228,7 @@ public interface FileTransfer extends Closeable {
             + "is invalid, the processor will not be invalid but will fail to change lastModifiedTime of the file.")
         .required(false)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .build();
     public static final PropertyDescriptor PERMISSIONS = new PropertyDescriptor.Builder()
         .name("Permissions")
@@ -237,7 +238,7 @@ public interface FileTransfer extends Closeable {
             + "fail to change permissions of the file.")
         .required(false)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .build();
     public static final PropertyDescriptor REMOTE_OWNER = new PropertyDescriptor.Builder()
         .name("Remote Owner")
@@ -246,7 +247,7 @@ public interface FileTransfer extends Closeable {
             + "will fail to change the owner of the file.")
         .required(false)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .build();
     public static final PropertyDescriptor REMOTE_GROUP = new PropertyDescriptor.Builder()
         .name("Remote Group")
@@ -255,7 +256,7 @@ public interface FileTransfer extends Closeable {
             + "will fail to change the group of the file.")
         .required(false)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .build();
     public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
         .name("Batch Size")

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java
index 1cee441..b203c31 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java
@@ -97,6 +97,7 @@ import org.apache.avro.io.DatumWriter;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.avro.AvroTypeUtil;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.util.StandardValidators;
 
 import javax.xml.bind.DatatypeConverter;
@@ -154,7 +155,7 @@ public class JdbcCommon {
                     + " 'Default Decimal Precision' is used when writing those undefined precision numbers.")
             .defaultValue(String.valueOf(DEFAULT_PRECISION_VALUE))
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(true)
             .build();
 
@@ -170,7 +171,7 @@ public class JdbcCommon {
                     + " e.g. 1.53 becomes 2 with scale 0, and 1.5 with scale 1.")
             .defaultValue(String.valueOf(DEFAULT_SCALE_VALUE))
             .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(true)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JmsProperties.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JmsProperties.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JmsProperties.java
index f538624..7a3ee0a 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JmsProperties.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JmsProperties.java
@@ -18,6 +18,7 @@ package org.apache.nifi.processors.standard.util;
 
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.ssl.SSLContextService;
 
@@ -155,14 +156,14 @@ public class JmsProperties {
             .description("The Priority of the Message")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     public static final PropertyDescriptor REPLY_TO_QUEUE = new PropertyDescriptor.Builder()
             .name("Reply-To Queue")
             .description("The name of the queue to which a reply to should be added")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
     public static final PropertyDescriptor MESSAGE_TTL = new PropertyDescriptor.Builder()
             .name("Message Time to Live")

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/SFTPTransfer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/SFTPTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/SFTPTransfer.java
index 99de60c..d7aa6e3 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/SFTPTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/SFTPTransfer.java
@@ -35,6 +35,7 @@ import java.util.regex.Pattern;
 
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.ProcessContext;
@@ -56,14 +57,14 @@ public class SFTPTransfer implements FileTransfer {
         .description("The fully qualified path to the Private Key file")
         .required(false)
         .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .build();
     public static final PropertyDescriptor PRIVATE_KEY_PASSPHRASE = new PropertyDescriptor.Builder()
         .name("Private Key Passphrase")
         .description("Password for the private key")
         .required(false)
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .sensitive(true)
         .build();
     public static final PropertyDescriptor HOST_KEY_FILE = new PropertyDescriptor.Builder()
@@ -83,7 +84,7 @@ public class SFTPTransfer implements FileTransfer {
         .name("Port")
         .description("The port that the remote system is listening on for file transfers")
         .addValidator(StandardValidators.PORT_VALIDATOR)
-        .expressionLanguageSupported(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
         .required(true)
         .defaultValue("22")
         .build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteStreamCommand.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteStreamCommand.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteStreamCommand.java
index 0988137..cdeec44 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteStreamCommand.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteStreamCommand.java
@@ -54,7 +54,6 @@ public class TestExecuteStreamCommand {
         String jarPath = exJar.getAbsolutePath();
         exJar.setExecutable(true);
         final TestRunner controller = TestRunners.newTestRunner(ExecuteStreamCommand.class);
-        controller.setValidateExpressionUsage(false);
         controller.enqueue(dummy.toPath());
         controller.setProperty(ExecuteStreamCommand.EXECUTION_COMMAND, "java");
         controller.setProperty(ExecuteStreamCommand.EXECUTION_ARGUMENTS, "-jar;" + jarPath);
@@ -87,7 +86,6 @@ public class TestExecuteStreamCommand {
         String jarPath = exJar.getAbsolutePath();
         exJar.setExecutable(true);
         final TestRunner controller = TestRunners.newTestRunner(ExecuteStreamCommand.class);
-        controller.setValidateExpressionUsage(false);
         controller.enqueue(dummy.toPath());
         controller.setProperty(ExecuteStreamCommand.EXECUTION_COMMAND, "java");
         controller.setProperty(ExecuteStreamCommand.EXECUTION_ARGUMENTS, "-jar;" + jarPath);
@@ -117,7 +115,6 @@ public class TestExecuteStreamCommand {
         String jarPath = exJar.getAbsolutePath();
         exJar.setExecutable(true);
         final TestRunner controller = TestRunners.newTestRunner(ExecuteStreamCommand.class);
-        controller.setValidateExpressionUsage(false);
         controller.enqueue(dummy10MBytes.toPath());
         controller.setProperty(ExecuteStreamCommand.EXECUTION_COMMAND, "java");
         controller.setProperty(ExecuteStreamCommand.EXECUTION_ARGUMENTS, "-jar;" + jarPath);
@@ -138,7 +135,6 @@ public class TestExecuteStreamCommand {
         String jarPath = exJar.getAbsolutePath();
         exJar.setExecutable(true);
         final TestRunner controller = TestRunners.newTestRunner(ExecuteStreamCommand.class);
-        controller.setValidateExpressionUsage(false);
         controller.enqueue(dummy.toPath());
         controller.setProperty(ExecuteStreamCommand.WORKING_DIR, "target");
         controller.setProperty(ExecuteStreamCommand.EXECUTION_COMMAND, "java");
@@ -161,7 +157,6 @@ public class TestExecuteStreamCommand {
         String jarPath = exJar.getAbsolutePath();
         exJar.setExecutable(true);
         final TestRunner controller = TestRunners.newTestRunner(ExecuteStreamCommand.class);
-        controller.setValidateExpressionUsage(false);
         controller.enqueue(dummy.toPath());
         controller.setProperty(ExecuteStreamCommand.WORKING_DIR, "target");
         controller.setProperty(ExecuteStreamCommand.EXECUTION_COMMAND, "java");
@@ -185,7 +180,6 @@ public class TestExecuteStreamCommand {
         testFile.delete();
         File dummy = new File("src/test/resources/ExecuteCommand/1000bytes.txt");
         final TestRunner controller = TestRunners.newTestRunner(ExecuteStreamCommand.class);
-        controller.setValidateExpressionUsage(false);
         controller.enqueue(dummy.toPath());
         controller.enqueue(dummy.toPath());
         controller.enqueue(dummy.toPath());
@@ -212,7 +206,6 @@ public class TestExecuteStreamCommand {
         final TestRunner controller = TestRunners.newTestRunner(ExecuteStreamCommand.class);
         controller.setProperty("NIFI_TEST_1", "testvalue1");
         controller.setProperty("NIFI_TEST_2", "testvalue2");
-        controller.setValidateExpressionUsage(false);
         controller.enqueue(dummy.toPath());
         controller.setProperty(ExecuteStreamCommand.WORKING_DIR, "target");
         controller.setProperty(ExecuteStreamCommand.EXECUTION_COMMAND, "java");
@@ -234,7 +227,6 @@ public class TestExecuteStreamCommand {
         File dummy = new File("src/test/resources/hello.txt");
         assertTrue(dummy.exists());
         final TestRunner controller = TestRunners.newTestRunner(ExecuteStreamCommand.class);
-        controller.setValidateExpressionUsage(false);
         controller.enqueue("".getBytes());
 
         if(isWindows()) {
@@ -268,7 +260,6 @@ public class TestExecuteStreamCommand {
         String jarPath = exJar.getAbsolutePath();
         exJar.setExecutable(true);
         final TestRunner controller = TestRunners.newTestRunner(ExecuteStreamCommand.class);
-        controller.setValidateExpressionUsage(false);
         controller.enqueue(dummy.toPath());
         controller.setProperty(ExecuteStreamCommand.EXECUTION_COMMAND, "java");
         controller.setProperty(ExecuteStreamCommand.EXECUTION_ARGUMENTS, "-jar;" + jarPath);
@@ -296,7 +287,6 @@ public class TestExecuteStreamCommand {
         String jarPath = exJar.getAbsolutePath();
         exJar.setExecutable(true);
         final TestRunner controller = TestRunners.newTestRunner(ExecuteStreamCommand.class);
-        controller.setValidateExpressionUsage(false);
         controller.enqueue("small test".getBytes());
         controller.setProperty(ExecuteStreamCommand.EXECUTION_COMMAND, "java");
         controller.setProperty(ExecuteStreamCommand.EXECUTION_ARGUMENTS, "-jar;" + jarPath);
@@ -335,7 +325,6 @@ public class TestExecuteStreamCommand {
         String jarPath = exJar.getAbsolutePath();
         exJar.setExecutable(true);
         final TestRunner controller = TestRunners.newTestRunner(ExecuteStreamCommand.class);
-        controller.setValidateExpressionUsage(false);
         controller.enqueue(dummy10MBytes.toPath());
         controller.setProperty(ExecuteStreamCommand.EXECUTION_COMMAND, "java");
         controller.setProperty(ExecuteStreamCommand.EXECUTION_ARGUMENTS, "-jar;" + jarPath);
@@ -361,7 +350,6 @@ public class TestExecuteStreamCommand {
         }
 
         final TestRunner controller = TestRunners.newTestRunner(ExecuteStreamCommand.class);
-        controller.setValidateExpressionUsage(false);
         controller.enqueue("".getBytes());
         if(isWindows()) {
             controller.setProperty(ExecuteStreamCommand.EXECUTION_COMMAND, "cmd.exe");
@@ -392,8 +380,6 @@ public class TestExecuteStreamCommand {
         String jarPath = exJar.getAbsolutePath();
         exJar.setExecutable(true);
         final TestRunner controller = TestRunners.newTestRunner(ExecuteStreamCommand.class);
-        controller.setValidateExpressionUsage(false);
-
         controller.enqueue(dummy.toPath());
         controller.setProperty(ExecuteStreamCommand.WORKING_DIR, "target");
         controller.setProperty(ExecuteStreamCommand.EXECUTION_COMMAND, "java");
@@ -415,7 +401,6 @@ public class TestExecuteStreamCommand {
         String jarPath = exJar.getAbsolutePath();
         exJar.setExecutable(true);
         final TestRunner controller = TestRunners.newTestRunner(ExecuteStreamCommand.class);
-        controller.setValidateExpressionUsage(false);
         controller.enqueue(dummy.toPath());
         controller.setProperty(ExecuteStreamCommand.WORKING_DIR, "target");
         controller.setProperty(ExecuteStreamCommand.EXECUTION_COMMAND, "java");
@@ -439,7 +424,6 @@ public class TestExecuteStreamCommand {
         final TestRunner controller = TestRunners.newTestRunner(ExecuteStreamCommand.class);
         controller.setProperty("NIFI_TEST_1", "testvalue1");
         controller.setProperty("NIFI_TEST_2", "testvalue2");
-        controller.setValidateExpressionUsage(false);
         controller.enqueue(dummy.toPath());
         controller.setProperty(ExecuteStreamCommand.WORKING_DIR, "target");
         controller.setProperty(ExecuteStreamCommand.EXECUTION_COMMAND, "java");
@@ -481,7 +465,6 @@ public class TestExecuteStreamCommand {
         String jarPath = exJar.getAbsolutePath();
         exJar.setExecutable(true);
         final TestRunner controller = TestRunners.newTestRunner(ExecuteStreamCommand.class);
-        controller.setValidateExpressionUsage(false);
         controller.enqueue(dummy.toPath());
         controller.setProperty(ExecuteStreamCommand.EXECUTION_COMMAND, "java");
         controller.setProperty(ExecuteStreamCommand.EXECUTION_ARGUMENTS, "-jar;" + jarPath);

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFTP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFTP.java
index 96a4236..450119f 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFTP.java
@@ -16,6 +16,16 @@
  */
 package org.apache.nifi.processors.standard;
 
+import static org.junit.Assert.assertEquals;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.ProcessContext;
@@ -36,16 +46,6 @@ import org.mockftpserver.fake.filesystem.FileEntry;
 import org.mockftpserver.fake.filesystem.FileSystem;
 import org.mockftpserver.fake.filesystem.WindowsFakeFileSystem;
 
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import static org.junit.Assert.assertEquals;
-
 public class TestFTP {
 
     final FakeFtpServer fakeFtpServer = new FakeFtpServer();
@@ -226,6 +226,7 @@ public class TestFTP {
 
         // Ensure wait for enough lag time.
         Thread.sleep(AbstractListProcessor.LISTING_LAG_MILLIS.get(TimeUnit.MILLISECONDS) * 2);
+
         runner.run();
 
         runner.assertTransferCount(FetchFTP.REL_SUCCESS, 1);

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetFile.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetFile.java
index baa8919..7ffdfd8 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetFile.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetFile.java
@@ -89,6 +89,7 @@ public class TestGetFile {
             runner.run();
             fail();
         } catch (AssertionError e) {
+            Throwable ex = e.getCause();
             assertTrue(e.getCause().getMessage()
                     .endsWith("does not have sufficient permissions (i.e., not writable and readable)"));
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestJoltTransformJSON.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestJoltTransformJSON.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestJoltTransformJSON.java
index 821aacc..14a2531 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestJoltTransformJSON.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestJoltTransformJSON.java
@@ -16,6 +16,8 @@
  */
 package org.apache.nifi.processors.standard;
 
+import static org.junit.Assert.assertTrue;
+
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.nio.file.Files;
@@ -35,8 +37,6 @@ import org.junit.Test;
 import com.bazaarvoice.jolt.Diffy;
 import com.bazaarvoice.jolt.JsonUtils;
 
-import static org.junit.Assert.assertTrue;
-
 public class TestJoltTransformJSON {
 
     final static Path JSON_INPUT = Paths.get("src/test/resources/TestJoltTransformJson/input.json");
@@ -118,7 +118,6 @@ public class TestJoltTransformJSON {
     @Test
     public void testCustomTransformationWithNoModule() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new JoltTransformJSON());
-        runner.setValidateExpressionUsage(false);
         final String spec = new String(Files.readAllBytes(Paths.get("src/test/resources/TestJoltTransformJson/customChainrSpec.json")));
         runner.setProperty(JoltTransformJSON.JOLT_SPEC, spec);
         runner.setProperty(JoltTransformJSON.CUSTOM_CLASS, "TestCustomJoltTransform");
@@ -169,7 +168,6 @@ public class TestJoltTransformJSON {
     @Test
     public void testTransformInputWithChainr() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new JoltTransformJSON());
-        runner.setValidateExpressionUsage(false);
         final String spec = new String(Files.readAllBytes(Paths.get("src/test/resources/TestJoltTransformJson/chainrSpec.json")));
         runner.setProperty(JoltTransformJSON.JOLT_SPEC, spec);
         runner.enqueue(JSON_INPUT);
@@ -186,7 +184,6 @@ public class TestJoltTransformJSON {
     @Test
     public void testTransformInputWithShiftr() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new JoltTransformJSON());
-        runner.setValidateExpressionUsage(false);
         final String spec = new String(Files.readAllBytes(Paths.get("src/test/resources/TestJoltTransformJson/shiftrSpec.json")));
         runner.setProperty(JoltTransformJSON.JOLT_SPEC, spec);
         runner.setProperty(JoltTransformJSON.JOLT_TRANSFORM, JoltTransformJSON.SHIFTR);
@@ -204,7 +201,6 @@ public class TestJoltTransformJSON {
     @Test
     public void testTransformInputWithDefaultr() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new JoltTransformJSON());
-        runner.setValidateExpressionUsage(false);
         final String spec = new String(Files.readAllBytes(Paths.get("src/test/resources/TestJoltTransformJson/defaultrSpec.json")));
         runner.setProperty(JoltTransformJSON.JOLT_SPEC, spec);
         runner.setProperty(JoltTransformJSON.JOLT_TRANSFORM, JoltTransformJSON.DEFAULTR);
@@ -220,7 +216,6 @@ public class TestJoltTransformJSON {
     @Test
     public void testTransformInputWithRemovr() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new JoltTransformJSON());
-        runner.setValidateExpressionUsage(false);
         final String spec = new String(Files.readAllBytes(Paths.get("src/test/resources/TestJoltTransformJson/removrSpec.json")));
         runner.setProperty(JoltTransformJSON.JOLT_SPEC, spec);
         runner.setProperty(JoltTransformJSON.JOLT_TRANSFORM, JoltTransformJSON.REMOVR);
@@ -236,7 +231,6 @@ public class TestJoltTransformJSON {
     @Test
     public void testTransformInputWithCardinality() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new JoltTransformJSON());
-        runner.setValidateExpressionUsage(false);
         final String spec = new String(Files.readAllBytes(Paths.get("src/test/resources/TestJoltTransformJson/cardrSpec.json")));
         runner.setProperty(JoltTransformJSON.JOLT_SPEC, spec);
         runner.setProperty(JoltTransformJSON.JOLT_TRANSFORM, JoltTransformJSON.CARDINALITY);
@@ -252,7 +246,6 @@ public class TestJoltTransformJSON {
     @Test
     public void testTransformInputWithSortr() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new JoltTransformJSON());
-        runner.setValidateExpressionUsage(false);
         runner.setProperty(JoltTransformJSON.JOLT_TRANSFORM, JoltTransformJSON.SORTR);
         runner.enqueue(JSON_INPUT);
         runner.run();
@@ -270,7 +263,6 @@ public class TestJoltTransformJSON {
     @Test
     public void testTransformInputWithDefaultrExpressionLanguage() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new JoltTransformJSON());
-        runner.setValidateExpressionUsage(false);
         final String spec = new String(Files.readAllBytes(Paths.get("src/test/resources/TestJoltTransformJson/defaultrELSpec.json")));
         runner.setProperty(JoltTransformJSON.JOLT_SPEC, spec);
         runner.setProperty(JoltTransformJSON.JOLT_TRANSFORM, JoltTransformJSON.DEFAULTR);
@@ -287,7 +279,6 @@ public class TestJoltTransformJSON {
     @Test
     public void testTransformInputWithModifierDefault() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new JoltTransformJSON());
-        runner.setValidateExpressionUsage(false);
         final String spec = new String(Files.readAllBytes(Paths.get("src/test/resources/TestJoltTransformJson/modifierDefaultSpec.json")));
         runner.setProperty(JoltTransformJSON.JOLT_SPEC, spec);
         runner.setProperty(JoltTransformJSON.JOLT_TRANSFORM, JoltTransformJSON.MODIFIER_DEFAULTR);
@@ -303,7 +294,6 @@ public class TestJoltTransformJSON {
     @Test
     public void testTransformInputWithModifierDefine() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new JoltTransformJSON());
-        runner.setValidateExpressionUsage(false);
         final String spec = new String(Files.readAllBytes(Paths.get("src/test/resources/TestJoltTransformJson/modifierDefineSpec.json")));
         runner.setProperty(JoltTransformJSON.JOLT_SPEC, spec);
         runner.setProperty(JoltTransformJSON.JOLT_TRANSFORM, JoltTransformJSON.MODIFIER_DEFAULTR);
@@ -319,7 +309,6 @@ public class TestJoltTransformJSON {
     @Test
     public void testTransformInputWithModifierOverwrite() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new JoltTransformJSON());
-        runner.setValidateExpressionUsage(false);
         final String spec = new String(Files.readAllBytes(Paths.get("src/test/resources/TestJoltTransformJson/modifierOverwriteSpec.json")));
         runner.setProperty(JoltTransformJSON.JOLT_SPEC, spec);
         runner.setProperty(JoltTransformJSON.JOLT_TRANSFORM, JoltTransformJSON.MODIFIER_DEFAULTR);
@@ -335,7 +324,6 @@ public class TestJoltTransformJSON {
     @Test
     public void testTransformInputWithSortrPopulatedSpec() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new JoltTransformJSON());
-        runner.setValidateExpressionUsage(false);
         runner.setProperty(JoltTransformJSON.JOLT_TRANSFORM, JoltTransformJSON.SORTR);
         runner.setProperty(JoltTransformJSON.JOLT_SPEC, "abcd");
         runner.enqueue(JSON_INPUT);
@@ -354,7 +342,6 @@ public class TestJoltTransformJSON {
     @Test
     public void testTransformInputWithCustomTransformationWithJar() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new JoltTransformJSON());
-        runner.setValidateExpressionUsage(false);
         final String customJarPath = "src/test/resources/TestJoltTransformJson/TestCustomJoltTransform.jar";
         final String spec = new String(Files.readAllBytes(Paths.get("src/test/resources/TestJoltTransformJson/chainrSpec.json")));
         runner.setProperty(JoltTransformJSON.JOLT_SPEC, spec);
@@ -375,7 +362,6 @@ public class TestJoltTransformJSON {
     @Test
     public void testTransformInputWithCustomTransformationWithDir() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new JoltTransformJSON());
-        runner.setValidateExpressionUsage(false);
         final String customJarPath = "src/test/resources/TestJoltTransformJson";
         final String spec = new String(Files.readAllBytes(Paths.get("src/test/resources/TestJoltTransformJson/chainrSpec.json")));
         runner.setProperty(JoltTransformJSON.JOLT_SPEC, spec);
@@ -396,7 +382,6 @@ public class TestJoltTransformJSON {
     @Test
     public void testTransformInputWithChainrEmbeddedCustomTransformation() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new JoltTransformJSON());
-        runner.setValidateExpressionUsage(false);
         final String customJarPath = "src/test/resources/TestJoltTransformJson";
         final String spec = new String(Files.readAllBytes(Paths.get("src/test/resources/TestJoltTransformJson/customChainrSpec.json")));
         runner.setProperty(JoltTransformJSON.JOLT_SPEC,spec);
@@ -415,7 +400,6 @@ public class TestJoltTransformJSON {
     @Test
     public void testTransformInputCustomTransformationIgnored() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new JoltTransformJSON());
-        runner.setValidateExpressionUsage(false);
         final String customJarPath = "src/test/resources/TestJoltTransformJson/TestCustomJoltTransform.jar";
         final String spec = new String(Files.readAllBytes(Paths.get("src/test/resources/TestJoltTransformJson/defaultrSpec.json")));
         runner.setProperty(JoltTransformJSON.JOLT_SPEC, spec);

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestLogMessage.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestLogMessage.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestLogMessage.java
index 98a8952..a07bfb1 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestLogMessage.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestLogMessage.java
@@ -17,6 +17,10 @@
 
 package org.apache.nifi.processors.standard;
 
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.util.MockComponentLog;
@@ -28,10 +32,6 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.List;
-
 public class TestLogMessage {
 
     private TestableLogMessage testableLogMessage;
@@ -75,8 +75,6 @@ public class TestLogMessage {
         flowAttributes.put("foobar", "baz");
 
         runner.enqueue("This is a message!", flowAttributes);
-        runner.setValidateExpressionUsage(false);
-
         runner.run();
 
         List<MockFlowFile> successFlowFiles = runner.getFlowFilesForRelationship(LogMessage.REL_SUCCESS);
@@ -106,8 +104,6 @@ public class TestLogMessage {
         flowAttributes.put("foobar", "baz");
 
         runner.enqueue("This is a message!", flowAttributes);
-        runner.setValidateExpressionUsage(false);
-
         runner.run();
 
         List<MockFlowFile> successFlowFiles = runner.getFlowFilesForRelationship(LogMessage.REL_SUCCESS);

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestMergeContent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestMergeContent.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestMergeContent.java
index 0af5f8b..d63d058 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestMergeContent.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestMergeContent.java
@@ -564,11 +564,20 @@ public class TestMergeContent {
         runner.setProperty(MergeContent.MAX_BIN_AGE, "1 sec");
         runner.setProperty(MergeContent.MERGE_FORMAT, MergeContent.MERGE_FORMAT_CONCAT);
         runner.setProperty(MergeContent.DELIMITER_STRATEGY, MergeContent.DELIMITER_STRATEGY_FILENAME);
-        runner.setProperty(MergeContent.HEADER, "src/test/resources/TestMergeContent/head");
-        runner.setProperty(MergeContent.DEMARCATOR, "src/test/resources/TestMergeContent/demarcate");
-        runner.setProperty(MergeContent.FOOTER, "src/test/resources/TestMergeContent/foot");
+        runner.setProperty(MergeContent.HEADER, "${header}");
+        runner.setProperty(MergeContent.DEMARCATOR, "${demarcator}");
+        runner.setProperty(MergeContent.FOOTER, "${footer}");
 
-        createFlowFiles(runner);
+
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put(CoreAttributes.MIME_TYPE.key(), "application/plain-text");
+        attributes.put("header", "src/test/resources/TestMergeContent/head");
+        attributes.put("demarcator", "src/test/resources/TestMergeContent/demarcate");
+        attributes.put("footer", "src/test/resources/TestMergeContent/foot");
+
+        runner.enqueue("Hello".getBytes("UTF-8"), attributes);
+        runner.enqueue(", ".getBytes("UTF-8"), attributes);
+        runner.enqueue("World!".getBytes("UTF-8"), attributes);
         runner.run();
 
         runner.assertQueueEmpty();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c787799/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryRecord.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryRecord.java
index 4e93253..345f8e4 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryRecord.java
@@ -57,6 +57,19 @@ public class TestQueryRecord {
 
     private static final String REL_NAME = "success";
 
+    public TestRunner getRunner() {
+        TestRunner runner = TestRunners.newTestRunner(QueryRecord.class);
+
+        /**
+         * we have to disable validation of expression language because the scope of the evaluation
+         * depends of the value of another property: if we are caching the schema/queries or not. If
+         * we don't disable the validation, it'll throw an error saying that the scope is incorrect.
+         */
+        runner.setValidateExpressionUsage(false);
+
+        return runner;
+    }
+
     @Test
     public void testSimple() throws InitializationException, IOException, SQLException {
         final MockRecordParser parser = new MockRecordParser();
@@ -66,7 +79,7 @@ public class TestQueryRecord {
 
         final MockRecordWriter writer = new MockRecordWriter("\"name\",\"points\"");
 
-        final TestRunner runner = TestRunners.newTestRunner(QueryRecord.class);
+        TestRunner runner = getRunner();
         runner.addControllerService("parser", parser);
         runner.enableControllerService(parser);
         runner.addControllerService("writer", writer);
@@ -99,7 +112,7 @@ public class TestQueryRecord {
 
         final MockRecordWriter writer = new MockRecordWriter("\"name\",\"points\"");
 
-        final TestRunner runner = TestRunners.newTestRunner(QueryRecord.class);
+        TestRunner runner = getRunner();
         runner.addControllerService("parser", parser);
         runner.enableControllerService(parser);
         runner.addControllerService("writer", writer);
@@ -139,7 +152,7 @@ public class TestQueryRecord {
 
         final MockRecordWriter writer = new MockRecordWriter("\"NAME\",\"POINTS\"");
 
-        final TestRunner runner = TestRunners.newTestRunner(QueryRecord.class);
+        TestRunner runner = getRunner();
         runner.addControllerService("parser", parser);
         runner.enableControllerService(parser);
         runner.addControllerService("writer", writer);
@@ -169,7 +182,7 @@ public class TestQueryRecord {
 
         final MockRecordWriter writer = new MockRecordWriter("\"name\",\"points\"");
 
-        final TestRunner runner = TestRunners.newTestRunner(QueryRecord.class);
+        TestRunner runner = getRunner();
         runner.enforceReadStreamsClosed(false);
         runner.addControllerService("parser", parser);
         runner.enableControllerService(parser);
@@ -200,7 +213,7 @@ public class TestQueryRecord {
 
         final MockRecordWriter writer = new MockRecordWriter("\"name\",\"points\"");
 
-        final TestRunner runner = TestRunners.newTestRunner(QueryRecord.class);
+        TestRunner runner = getRunner();
         runner.addControllerService("parser", parser);
         runner.enableControllerService(parser);
         runner.addControllerService("writer", writer);
@@ -235,7 +248,7 @@ public class TestQueryRecord {
         colNames.add("FAV_GREETING");
         final ResultSetValidatingRecordWriter writer = new ResultSetValidatingRecordWriter(colNames);
 
-        final TestRunner runner = TestRunners.newTestRunner(QueryRecord.class);
+        TestRunner runner = getRunner();
         runner.addControllerService("parser", parser);
         runner.enableControllerService(parser);
         runner.addControllerService("writer", writer);