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:20 UTC

[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

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();