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

[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

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)