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

[1/9] nifi git commit: NIFI-4224: - Initial implementation of Process Group level Variable Registry - Updated to incorporate PR Feedback - Changed log message because slf4j-simple apparently has a memory leak; passing a String instead of passing in the C

Repository: nifi
Updated Branches:
  refs/heads/master c1b99d584 -> 5cd8e93be


http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java
index 9af57b6..258af72 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java
@@ -16,6 +16,13 @@
  */
 package org.apache.nifi.web.dao.impl;
 
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Future;
+
 import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.ConnectableType;
 import org.apache.nifi.connectable.Port;
@@ -23,14 +30,15 @@ import org.apache.nifi.connectable.Position;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
 import org.apache.nifi.controller.ScheduledState;
+import org.apache.nifi.controller.service.ControllerServiceNode;
+import org.apache.nifi.controller.service.ControllerServiceState;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.web.ResourceNotFoundException;
 import org.apache.nifi.web.api.dto.ProcessGroupDTO;
+import org.apache.nifi.web.api.dto.VariableRegistryDTO;
+import org.apache.nifi.web.api.entity.VariableEntity;
 import org.apache.nifi.web.dao.ProcessGroupDAO;
 
-import java.util.HashSet;
-import java.util.Set;
-
 public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGroupDAO {
 
     private FlowController flowController;
@@ -64,6 +72,10 @@ public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGrou
     }
 
     @Override
+    public void verifyUpdate(final ProcessGroupDTO processGroup) {
+    }
+
+    @Override
     public ProcessGroup getProcessGroup(String groupId) {
         return locateProcessGroup(flowController, groupId);
     }
@@ -99,14 +111,32 @@ public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGrou
     }
 
     @Override
-    public void scheduleComponents(final String groupId, final ScheduledState state, final Set<String> componentIds) {
+    public void verifyActivateControllerServices(final String groupId, final ControllerServiceState state, final Set<String> serviceIds) {
         final ProcessGroup group = locateProcessGroup(flowController, groupId);
 
+        group.findAllControllerServices().stream()
+            .filter(service -> serviceIds.contains(service.getIdentifier()))
+            .forEach(service -> {
+                if (state == ControllerServiceState.ENABLED) {
+                    service.verifyCanEnable();
+                } else {
+                    service.verifyCanDisable();
+                }
+            });
+    }
+
+    @Override
+    public CompletableFuture<Void> scheduleComponents(final String groupId, final ScheduledState state, final Set<String> componentIds) {
+        final ProcessGroup group = locateProcessGroup(flowController, groupId);
+
+        CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
+
         for (final String componentId : componentIds) {
             final Connectable connectable = group.findLocalConnectable(componentId);
             if (ScheduledState.RUNNING.equals(state)) {
                 if (ConnectableType.PROCESSOR.equals(connectable.getConnectableType())) {
-                    connectable.getProcessGroup().startProcessor((ProcessorNode) connectable);
+                    final CompletableFuture<?> processorFuture = connectable.getProcessGroup().startProcessor((ProcessorNode) connectable);
+                    future = CompletableFuture.allOf(future, processorFuture);
                 } else if (ConnectableType.INPUT_PORT.equals(connectable.getConnectableType())) {
                     connectable.getProcessGroup().startInputPort((Port) connectable);
                 } else if (ConnectableType.OUTPUT_PORT.equals(connectable.getConnectableType())) {
@@ -114,7 +144,8 @@ public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGrou
                 }
             } else {
                 if (ConnectableType.PROCESSOR.equals(connectable.getConnectableType())) {
-                    connectable.getProcessGroup().stopProcessor((ProcessorNode) connectable);
+                    final CompletableFuture<?> processorFuture = connectable.getProcessGroup().stopProcessor((ProcessorNode) connectable);
+                    future = CompletableFuture.allOf(future, processorFuture);
                 } else if (ConnectableType.INPUT_PORT.equals(connectable.getConnectableType())) {
                     connectable.getProcessGroup().stopInputPort((Port) connectable);
                 } else if (ConnectableType.OUTPUT_PORT.equals(connectable.getConnectableType())) {
@@ -122,6 +153,27 @@ public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGrou
                 }
             }
         }
+
+        return future;
+    }
+
+    @Override
+    public Future<Void> activateControllerServices(final String groupId, final ControllerServiceState state, final Set<String> serviceIds) {
+        final ProcessGroup group = locateProcessGroup(flowController, groupId);
+
+        CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
+        for (final String serviceId : serviceIds) {
+            final ControllerServiceNode serviceNode = group.findControllerService(serviceId);
+            if (ControllerServiceState.ENABLED.equals(state)) {
+                final CompletableFuture<Void> serviceFuture = flowController.enableControllerService(serviceNode);
+                future = CompletableFuture.allOf(future, serviceFuture);
+            } else {
+                final CompletableFuture<Void> serviceFuture = flowController.disableControllerService(serviceNode);
+                future = CompletableFuture.allOf(future, serviceFuture);
+            }
+        }
+
+        return future;
     }
 
     @Override
@@ -145,6 +197,22 @@ public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGrou
     }
 
     @Override
+    public ProcessGroup updateVariableRegistry(final VariableRegistryDTO variableRegistry) {
+        final ProcessGroup group = locateProcessGroup(flowController, variableRegistry.getProcessGroupId());
+        if (group == null) {
+            throw new ResourceNotFoundException("Could not find Process Group with ID " + variableRegistry.getProcessGroupId());
+        }
+
+        final Map<String, String> variableMap = new HashMap<>();
+        variableRegistry.getVariables().stream() // have to use forEach here instead of using Collectors.toMap because value may be null
+            .map(VariableEntity::getVariable)
+            .forEach(var -> variableMap.put(var.getName(), var.getValue()));
+
+        group.setVariables(variableMap);
+        return group;
+    }
+
+    @Override
     public void verifyDelete(String groupId) {
         ProcessGroup group = locateProcessGroup(flowController, groupId);
         group.verifyCanDelete();

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/Pause.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/Pause.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/Pause.java
new file mode 100644
index 0000000..c2ef890
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/Pause.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.web.util;
+
+public interface Pause {
+
+    /**
+     * Waits up to given amount of time, and returns <code>true</code> if the action being performed
+     * should continue, <code>false</code> if the action being performed has been cancelled
+     *
+     * @return <code>true</code> if the action should continue, <code>false</code> otherwise
+     */
+    boolean pause();
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
index 2311b07..8ee39f0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
@@ -284,6 +284,7 @@
         <property name="requestReplicator" ref="requestReplicator" />
         <property name="authorizer" ref="authorizer"/>
         <property name="flowController" ref="flowController" />
+        <property name="dtoFactory" ref="dtoFactory" />
     </bean>
     <bean id="processorResource" class="org.apache.nifi.web.api.ProcessorResource" scope="singleton">
         <property name="serviceFacade" ref="serviceFacade"/>


[2/9] nifi git commit: NIFI-4224: - Initial implementation of Process Group level Variable Registry - Updated to incorporate PR Feedback - Changed log message because slf4j-simple apparently has a memory leak; passing a String instead of passing in the C

Posted by mc...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
index a9167ae..0b634a1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
@@ -16,14 +16,54 @@
  */
 package org.apache.nifi.web.api;
 
-import com.sun.jersey.api.core.ResourceContext;
-import com.sun.jersey.multipart.FormDataParam;
-import com.wordnik.swagger.annotations.Api;
-import com.wordnik.swagger.annotations.ApiOperation;
-import com.wordnik.swagger.annotations.ApiParam;
-import com.wordnik.swagger.annotations.ApiResponse;
-import com.wordnik.swagger.annotations.ApiResponses;
-import com.wordnik.swagger.annotations.Authorization;
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.HttpMethod;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.Response.Status;
+import javax.ws.rs.core.UriBuilder;
+import javax.ws.rs.core.UriInfo;
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBElement;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Unmarshaller;
+import javax.xml.transform.stream.StreamSource;
+
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.AuthorizableLookup;
 import org.apache.nifi.authorization.AuthorizeControllerServiceReference;
@@ -38,26 +78,41 @@ import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.authorization.user.NiFiUserUtils;
 import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.connectable.ConnectableType;
+import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.serialization.FlowEncodingVersion;
+import org.apache.nifi.controller.service.ControllerServiceState;
+import org.apache.nifi.framework.security.util.SslContextFactory;
+import org.apache.nifi.registry.variable.VariableRegistryUpdateRequest;
+import org.apache.nifi.registry.variable.VariableRegistryUpdateStep;
 import org.apache.nifi.remote.util.SiteToSiteRestApiClient;
 import org.apache.nifi.util.BundleUtils;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.NiFiServiceFacade;
 import org.apache.nifi.web.ResourceNotFoundException;
 import org.apache.nifi.web.Revision;
+import org.apache.nifi.web.api.dto.AffectedComponentDTO;
 import org.apache.nifi.web.api.dto.BundleDTO;
 import org.apache.nifi.web.api.dto.ConnectionDTO;
 import org.apache.nifi.web.api.dto.ControllerServiceDTO;
+import org.apache.nifi.web.api.dto.DtoFactory;
 import org.apache.nifi.web.api.dto.FlowSnippetDTO;
 import org.apache.nifi.web.api.dto.PositionDTO;
 import org.apache.nifi.web.api.dto.ProcessGroupDTO;
 import org.apache.nifi.web.api.dto.ProcessorConfigDTO;
 import org.apache.nifi.web.api.dto.ProcessorDTO;
 import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
+import org.apache.nifi.web.api.dto.RevisionDTO;
 import org.apache.nifi.web.api.dto.TemplateDTO;
+import org.apache.nifi.web.api.dto.VariableRegistryDTO;
 import org.apache.nifi.web.api.dto.flow.FlowDTO;
+import org.apache.nifi.web.api.dto.status.ProcessGroupStatusDTO;
+import org.apache.nifi.web.api.dto.status.ProcessGroupStatusSnapshotDTO;
+import org.apache.nifi.web.api.entity.ActivateControllerServicesEntity;
 import org.apache.nifi.web.api.entity.ConnectionEntity;
 import org.apache.nifi.web.api.entity.ConnectionsEntity;
 import org.apache.nifi.web.api.entity.ControllerServiceEntity;
+import org.apache.nifi.web.api.entity.ControllerServicesEntity;
 import org.apache.nifi.web.api.entity.CopySnippetRequestEntity;
 import org.apache.nifi.web.api.entity.CreateTemplateRequestEntity;
 import org.apache.nifi.web.api.entity.FlowEntity;
@@ -70,47 +125,36 @@ import org.apache.nifi.web.api.entity.LabelsEntity;
 import org.apache.nifi.web.api.entity.OutputPortsEntity;
 import org.apache.nifi.web.api.entity.PortEntity;
 import org.apache.nifi.web.api.entity.ProcessGroupEntity;
+import org.apache.nifi.web.api.entity.ProcessGroupStatusEntity;
+import org.apache.nifi.web.api.entity.ProcessGroupStatusSnapshotEntity;
 import org.apache.nifi.web.api.entity.ProcessGroupsEntity;
 import org.apache.nifi.web.api.entity.ProcessorEntity;
 import org.apache.nifi.web.api.entity.ProcessorsEntity;
 import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
 import org.apache.nifi.web.api.entity.RemoteProcessGroupsEntity;
+import org.apache.nifi.web.api.entity.ScheduleComponentsEntity;
 import org.apache.nifi.web.api.entity.TemplateEntity;
+import org.apache.nifi.web.api.entity.VariableRegistryEntity;
+import org.apache.nifi.web.api.entity.VariableRegistryUpdateRequestEntity;
 import org.apache.nifi.web.api.request.ClientIdParameter;
 import org.apache.nifi.web.api.request.LongParameter;
+import org.apache.nifi.web.util.Pause;
+import org.apache.nifi.web.util.WebUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.servlet.http.HttpServletRequest;
-import javax.ws.rs.Consumes;
-import javax.ws.rs.DELETE;
-import javax.ws.rs.DefaultValue;
-import javax.ws.rs.GET;
-import javax.ws.rs.HttpMethod;
-import javax.ws.rs.POST;
-import javax.ws.rs.PUT;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.Produces;
-import javax.ws.rs.QueryParam;
-import javax.ws.rs.core.Context;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.Response;
-import javax.ws.rs.core.UriBuilder;
-import javax.ws.rs.core.UriInfo;
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBElement;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Unmarshaller;
-import javax.xml.transform.stream.StreamSource;
-import java.io.InputStream;
-import java.net.URI;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.function.Consumer;
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.config.ClientConfig;
+import com.sun.jersey.api.client.config.DefaultClientConfig;
+import com.sun.jersey.api.core.ResourceContext;
+import com.sun.jersey.multipart.FormDataParam;
+import com.wordnik.swagger.annotations.Api;
+import com.wordnik.swagger.annotations.ApiOperation;
+import com.wordnik.swagger.annotations.ApiParam;
+import com.wordnik.swagger.annotations.ApiResponse;
+import com.wordnik.swagger.annotations.ApiResponses;
+import com.wordnik.swagger.annotations.Authorization;
 
 /**
  * RESTful endpoint for managing a Group.
@@ -139,6 +183,22 @@ public class ProcessGroupResource extends ApplicationResource {
     private ConnectionResource connectionResource;
     private TemplateResource templateResource;
     private ControllerServiceResource controllerServiceResource;
+    private DtoFactory dtoFactory;
+
+    private final ConcurrentMap<String, VariableRegistryUpdateRequest> varRegistryUpdateRequests = new ConcurrentHashMap<>();
+    private static final int MAX_VARIABLE_REGISTRY_UPDATE_REQUESTS = 100;
+    private static final long VARIABLE_REGISTRY_UPDATE_REQUEST_EXPIRATION = TimeUnit.MINUTES.toMillis(1L);
+    private final ExecutorService variableRegistryThreadPool = new ThreadPoolExecutor(1, 50, 5L, TimeUnit.SECONDS,
+        new ArrayBlockingQueue<Runnable>(MAX_VARIABLE_REGISTRY_UPDATE_REQUESTS),
+        new ThreadFactory() {
+            @Override
+            public Thread newThread(final Runnable r) {
+                final Thread thread = Executors.defaultThreadFactory().newThread(r);
+                thread.setName("Variable Registry Update Thread");
+                thread.setDaemon(true);
+                return thread;
+            }
+        });
 
     /**
      * Populates the remaining fields in the specified process groups.
@@ -164,6 +224,7 @@ public class ProcessGroupResource extends ApplicationResource {
         return processGroupEntity;
     }
 
+
     /**
      * Populates the remaining content of the specified snippet.
      */
@@ -238,6 +299,49 @@ public class ProcessGroupResource extends ApplicationResource {
         return generateOkResponse(entity).build();
     }
 
+
+    /**
+     * Retrieves the Variable Registry for the group with the given ID
+     *
+     * @param groupId the ID of the Process Group
+     * @return the Variable Registry for the group
+     */
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/variable-registry")
+    @ApiOperation(value = "Gets a process group's variable registry",
+        response = VariableRegistryEntity.class,
+        notes = NON_GUARANTEED_ENDPOINT,
+        authorizations = {
+            @Authorization(value = "Read - /process-groups/{uuid}", type = "")
+        })
+    @ApiResponses(value = {
+        @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+        @ApiResponse(code = 401, message = "Client could not be authenticated."),
+        @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+        @ApiResponse(code = 404, message = "The specified resource could not be found."),
+        @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+    })
+    public Response getVariableRegistry(
+        @ApiParam(value = "The process group id.", required = true) @PathParam("id") final String groupId,
+        @ApiParam(value = "Whether or not to include ancestor groups", required = false) @QueryParam("includeAncestorGroups") @DefaultValue("true") final boolean includeAncestorGroups) {
+
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.GET);
+        }
+
+        // authorize access
+        serviceFacade.authorizeAccess(lookup -> {
+            final Authorizable processGroup = lookup.getProcessGroup(groupId).getAuthorizable();
+            processGroup.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
+        });
+
+        // get this process group's variable registry
+        final VariableRegistryEntity entity = serviceFacade.getVariableRegistry(groupId, includeAncestorGroups);
+        return generateOkResponse(entity).build();
+    }
+
     /**
      * Updates the specified process group.
      *
@@ -314,7 +418,7 @@ public class ProcessGroupResource extends ApplicationResource {
                     Authorizable authorizable = lookup.getProcessGroup(id).getAuthorizable();
                     authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
                 },
-                null,
+                () -> serviceFacade.verifyUpdateProcessGroup(requestProcessGroupDTO),
                 (revision, processGroupEntity) -> {
                     // update the process group
                     final ProcessGroupEntity entity = serviceFacade.updateProcessGroup(revision, processGroupEntity.getComponent());
@@ -325,6 +429,854 @@ public class ProcessGroupResource extends ApplicationResource {
         );
     }
 
+
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{groupId}/variable-registry/update-requests/{updateId}")
+    @ApiOperation(value = "Gets a process group's variable registry",
+        response = VariableRegistryUpdateRequestEntity.class,
+        notes = NON_GUARANTEED_ENDPOINT,
+        authorizations = {
+            @Authorization(value = "Read - /process-groups/{uuid}", type = "")
+        })
+    @ApiResponses(value = {
+        @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+        @ApiResponse(code = 401, message = "Client could not be authenticated."),
+        @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+        @ApiResponse(code = 404, message = "The specified resource could not be found."),
+        @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+    })
+    public Response getVariableRegistryUpdateRequest(
+        @ApiParam(value = "The process group id.", required = true) @PathParam("groupId") final String groupId,
+        @ApiParam(value = "The ID of the Variable Registry Update Request", required = true) @PathParam("updateId") final String updateId) {
+
+        if (groupId == null || updateId == null) {
+            throw new IllegalArgumentException("Group ID and Update ID must both be specified.");
+        }
+
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.GET);
+        }
+
+        // authorize access
+        serviceFacade.authorizeAccess(lookup -> {
+            final Authorizable processGroup = lookup.getProcessGroup(groupId).getAuthorizable();
+            processGroup.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
+        });
+
+        final VariableRegistryUpdateRequest request = varRegistryUpdateRequests.get(updateId);
+        if (request == null) {
+            throw new ResourceNotFoundException("Could not find a Variable Registry Update Request with identifier " + updateId);
+        }
+
+        if (!groupId.equals(request.getProcessGroupId())) {
+            throw new ResourceNotFoundException("Could not find a Variable Registry Update Request with identifier " + updateId + " for Process Group with identifier " + groupId);
+        }
+
+        final VariableRegistryUpdateRequestEntity entity = new VariableRegistryUpdateRequestEntity();
+        entity.setRequestDto(dtoFactory.createVariableRegistryUpdateRequestDto(request));
+        entity.getRequestDto().setUri(generateResourceUri("process-groups", groupId, "variable-registry", updateId));
+        return generateOkResponse(entity).build();
+    }
+
+
+    @DELETE
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{groupId}/variable-registry/update-requests/{updateId}")
+    @ApiOperation(value = "Deletes an update request for a process group's variable registry. If the request is not yet complete, it will automatically be cancelled.",
+        response = VariableRegistryUpdateRequestEntity.class,
+        notes = NON_GUARANTEED_ENDPOINT,
+        authorizations = {
+            @Authorization(value = "Read - /process-groups/{uuid}", type = "")
+        })
+    @ApiResponses(value = {
+        @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+        @ApiResponse(code = 401, message = "Client could not be authenticated."),
+        @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+        @ApiResponse(code = 404, message = "The specified resource could not be found."),
+        @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+    })
+    public Response deleteVariableRegistryUpdateRequest(
+        @ApiParam(value = "The process group id.", required = true) @PathParam("groupId") final String groupId,
+        @ApiParam(value = "The ID of the Variable Registry Update Request", required = true) @PathParam("updateId") final String updateId) {
+
+        if (groupId == null || updateId == null) {
+            throw new IllegalArgumentException("Group ID and Update ID must both be specified.");
+        }
+
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.DELETE);
+        }
+
+        // authorize access
+        serviceFacade.authorizeAccess(lookup -> {
+            final Authorizable processGroup = lookup.getProcessGroup(groupId).getAuthorizable();
+            processGroup.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
+            processGroup.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+        });
+
+        final VariableRegistryUpdateRequest request = varRegistryUpdateRequests.remove(updateId);
+        if (request == null) {
+            throw new ResourceNotFoundException("Could not find a Variable Registry Update Request with identifier " + updateId);
+        }
+
+        if (!groupId.equals(request.getProcessGroupId())) {
+            throw new ResourceNotFoundException("Could not find a Variable Registry Update Request with identifier " + updateId + " for Process Group with identifier " + groupId);
+        }
+
+        request.cancel();
+
+        final VariableRegistryUpdateRequestEntity entity = new VariableRegistryUpdateRequestEntity();
+        entity.setRequestDto(dtoFactory.createVariableRegistryUpdateRequestDto(request));
+        entity.getRequestDto().setUri(generateResourceUri("process-groups", groupId, "variable-registry", updateId));
+        return generateOkResponse(entity).build();
+    }
+
+
+    @PUT
+    @Consumes(MediaType.APPLICATION_JSON)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/variable-registry")
+    @ApiOperation(value = "Updates the contents of a Process Group's variable Registry", response = VariableRegistryEntity.class, notes = NON_GUARANTEED_ENDPOINT, authorizations = {
+        @Authorization(value = "Write - /process-groups/{uuid}", type = "")
+    })
+    @ApiResponses(value = {
+        @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+        @ApiResponse(code = 401, message = "Client could not be authenticated."),
+        @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+        @ApiResponse(code = 404, message = "The specified resource could not be found."),
+        @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+    })
+    public Response updateVariableRegistry(
+        @Context final HttpServletRequest httpServletRequest,
+        @ApiParam(value = "The process group id.", required = true) @PathParam("id") final String groupId,
+        @ApiParam(value = "The process group configuration details.", required = true) final VariableRegistryEntity requestEntity) {
+
+        if (requestEntity == null || requestEntity.getVariableRegistry() == null) {
+            throw new IllegalArgumentException("Variable Registry details must be specified.");
+        }
+
+        if (requestEntity.getProcessGroupRevision() == null) {
+            throw new IllegalArgumentException("Process Group Revision must be specified.");
+        }
+
+        // ensure the same id is being used
+        final VariableRegistryDTO registryDto = requestEntity.getVariableRegistry();
+        if (!groupId.equals(registryDto.getProcessGroupId())) {
+            throw new IllegalArgumentException(String.format("The process group id (%s) in the request body does "
+                + "not equal the process group id of the requested resource (%s).", registryDto.getProcessGroupId(), groupId));
+        }
+
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.PUT, requestEntity);
+        }
+
+        // handle expects request (usually from the cluster manager)
+        final Revision requestRevision = getRevision(requestEntity.getProcessGroupRevision(), groupId);
+        return withWriteLock(
+            serviceFacade,
+            requestEntity,
+            requestRevision,
+            lookup -> {
+                Authorizable authorizable = lookup.getProcessGroup(groupId).getAuthorizable();
+                authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+            },
+            null,
+            (revision, processGroupEntity) -> {
+                // update the process group
+                final VariableRegistryEntity entity = serviceFacade.updateVariableRegistry(revision, registryDto);
+                return generateOkResponse(entity).build();
+            });
+    }
+
+
+    /**
+     * Updates the variable registry for the specified process group.
+     *
+     * @param httpServletRequest request
+     * @param groupId The id of the process group.
+     * @param requestEntity the Variable Registry Entity
+     * @return A Variable Registry Entry.
+     */
+    @POST
+    @Consumes(MediaType.APPLICATION_JSON)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/variable-registry/update-requests")
+    @ApiOperation(value = "Submits a request to update a process group's variable registry",
+        response = VariableRegistryUpdateRequestEntity.class,
+        notes = NON_GUARANTEED_ENDPOINT,
+        authorizations = {
+            @Authorization(value = "Write - /process-groups/{uuid}", type = "")
+        })
+    @ApiResponses(value = {
+        @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+        @ApiResponse(code = 401, message = "Client could not be authenticated."),
+        @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+        @ApiResponse(code = 404, message = "The specified resource could not be found."),
+        @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+    })
+    public Response submitUpdateVariableRegistryRequest(
+        @Context final HttpServletRequest httpServletRequest,
+        @ApiParam(value = "The process group id.", required = true) @PathParam("id") final String groupId,
+        @ApiParam(value = "The process group configuration details.", required = true) final VariableRegistryEntity requestEntity) {
+
+        if (requestEntity == null || requestEntity.getVariableRegistry() == null) {
+            throw new IllegalArgumentException("Variable Registry details must be specified.");
+        }
+
+        if (requestEntity.getProcessGroupRevision() == null) {
+            throw new IllegalArgumentException("Process Group Revision must be specified.");
+        }
+
+        // In order to update variables in a variable registry, we have to perform the following steps:
+        // 1. Determine Affected Components (this includes any Processors and Controller Services and any components that reference an affected Controller Service).
+        //    1a. Determine ID's of components
+        //    1b. Determine Revision's of associated components
+        // 2. Stop All Affected Processors
+        // 3. Disable All Affected Controller Services
+        // 4. Update the Variables
+        // 5. Re-Enable all Affected Controller Services (services only, not dependent components)
+        // 6. Re-Enable all Processors that Depended on the Controller Services
+
+        // Determine the affected components (and their associated revisions)
+        final VariableRegistryEntity computedEntity = serviceFacade.populateAffectedComponents(requestEntity.getVariableRegistry());
+        final VariableRegistryDTO computedRegistryDto = computedEntity.getVariableRegistry();
+        if (computedRegistryDto == null) {
+            throw new ResourceNotFoundException(String.format("Unable to locate group with id '%s'.", groupId));
+        }
+
+        final Set<AffectedComponentDTO> affectedComponents = serviceFacade.getComponentsAffectedByVariableRegistryUpdate(requestEntity.getVariableRegistry());
+
+        final Map<String, List<AffectedComponentDTO>> affectedComponentsByType = affectedComponents.stream()
+            .collect(Collectors.groupingBy(comp -> comp.getComponentType()));
+
+        final List<AffectedComponentDTO> affectedProcessors = affectedComponentsByType.get(AffectedComponentDTO.COMPONENT_TYPE_PROCESSOR);
+        final List<AffectedComponentDTO> affectedServices = affectedComponentsByType.get(AffectedComponentDTO.COMPONENT_TYPE_CONTROLLER_SERVICE);
+
+
+        if (isReplicateRequest()) {
+            // update the variable registry
+            final VariableRegistryUpdateRequest updateRequest = createVariableRegistryUpdateRequest(groupId);
+            updateRequest.getIdentifyRelevantComponentsStep().setComplete(true);
+            final URI originalUri = getAbsolutePath();
+
+            // Submit the task to be run in the background
+            final Runnable taskWrapper = () -> {
+                try {
+                    updateVariableRegistryReplicated(groupId, originalUri, affectedProcessors, affectedServices, updateRequest, requestEntity);
+                } catch (final Exception e) {
+                    logger.error("Failed to update variable registry", e);
+                    updateRequest.setFailureReason("An unexpected error has occurred: " + e);
+                }
+            };
+
+            variableRegistryThreadPool.submit(taskWrapper);
+
+            final VariableRegistryUpdateRequestEntity responseEntity = new VariableRegistryUpdateRequestEntity();
+            responseEntity.setRequestDto(dtoFactory.createVariableRegistryUpdateRequestDto(updateRequest));
+            responseEntity.getRequestDto().setUri(generateResourceUri("process-groups", groupId, "variable-registry", "update-requests", updateRequest.getRequestId()));
+
+            final URI location = URI.create(responseEntity.getRequestDto().getUri());
+            return Response.status(Status.ACCEPTED).location(location).entity(responseEntity).build();
+        }
+
+
+        final Revision requestRevision = getRevision(requestEntity.getProcessGroupRevision(), groupId);
+        return withWriteLock(
+            serviceFacade,
+            requestEntity,
+            requestRevision,
+            lookup -> {
+                final NiFiUser user = NiFiUserUtils.getNiFiUser();
+
+                final Authorizable groupAuthorizable = lookup.getProcessGroup(groupId).getAuthorizable();
+                groupAuthorizable.authorize(authorizer, RequestAction.WRITE, user);
+
+                // For every component that is affected, the user must have READ permissions and WRITE permissions
+                // (because this action requires stopping the component).
+                if (affectedProcessors != null) {
+                    for (final AffectedComponentDTO affectedComponent : affectedProcessors) {
+                        final Authorizable authorizable = lookup.getProcessor(affectedComponent.getComponentId()).getAuthorizable();
+                        authorizable.authorize(authorizer, RequestAction.READ, user);
+                        authorizable.authorize(authorizer, RequestAction.WRITE, user);
+                    }
+                }
+
+                if (affectedServices != null) {
+                    for (final AffectedComponentDTO affectedComponent : affectedServices) {
+                        final Authorizable authorizable = lookup.getControllerService(affectedComponent.getComponentId()).getAuthorizable();
+                        authorizable.authorize(authorizer, RequestAction.READ, user);
+                        authorizable.authorize(authorizer, RequestAction.WRITE, user);
+                    }
+                }
+            },
+            null,
+            (revision, varRegistryEntity) -> {
+                return updateVariableRegistryLocal(groupId, affectedProcessors, affectedServices, requestEntity);
+            });
+    }
+
+    private Pause createPause(final VariableRegistryUpdateRequest updateRequest) {
+        return new Pause() {
+            @Override
+            public boolean pause() {
+                if (updateRequest.isComplete()) {
+                    return false;
+                }
+
+                try {
+                    Thread.sleep(500);
+                } catch (final InterruptedException ie) {
+                    Thread.currentThread().interrupt();
+                    return false;
+                }
+
+                return !updateRequest.isComplete();
+            }
+        };
+    }
+
+    private void updateVariableRegistryReplicated(final String groupId, final URI originalUri, final Collection<AffectedComponentDTO> affectedProcessors,
+        final Collection<AffectedComponentDTO> affectedServices,
+        final VariableRegistryUpdateRequest updateRequest, final VariableRegistryEntity requestEntity) {
+
+        final NiFiProperties properties = getProperties();
+        final Client jerseyClient = WebUtils.createClient(new DefaultClientConfig(), SslContextFactory.createSslContext(properties));
+        final int connectionTimeout = (int) FormatUtils.getTimeDuration(properties.getClusterNodeConnectionTimeout(), TimeUnit.MILLISECONDS);
+        final int readTimeout = (int) FormatUtils.getTimeDuration(properties.getClusterNodeReadTimeout(), TimeUnit.MILLISECONDS);
+        jerseyClient.getProperties().put(ClientConfig.PROPERTY_CONNECT_TIMEOUT, connectionTimeout);
+        jerseyClient.getProperties().put(ClientConfig.PROPERTY_READ_TIMEOUT, readTimeout);
+        jerseyClient.getProperties().put(ClientConfig.PROPERTY_FOLLOW_REDIRECTS, Boolean.TRUE);
+
+        final Pause pause = createPause(updateRequest);
+
+        // stop processors
+        if (affectedProcessors != null) {
+            logger.info("In order to update Variable Registry for Process Group with ID {}, "
+                + "replicating request to stop {} affected processors", groupId, affectedProcessors.size());
+
+            scheduleProcessors(groupId, originalUri, jerseyClient, updateRequest, pause,
+                affectedProcessors, ScheduledState.STOPPED, updateRequest.getStopProcessorsStep());
+        }
+
+        // disable controller services
+        if (affectedServices != null) {
+            logger.info("In order to update Variable Registry for Process Group with ID {}, "
+                + "replicating request to stop {} affected Controller Services", groupId, affectedServices.size());
+
+            activateControllerServices(groupId, originalUri, jerseyClient, updateRequest, pause,
+                affectedServices, ControllerServiceState.DISABLED, updateRequest.getDisableServicesStep());
+        }
+
+        // apply updates
+        logger.info("In order to update Variable Registry for Process Group with ID {}, "
+            + "replicating request to apply updates to variable registry", groupId);
+        applyVariableRegistryUpdate(groupId, originalUri, jerseyClient, updateRequest, requestEntity);
+
+        // re-enable controller services
+        if (affectedServices != null) {
+            logger.info("In order to update Variable Registry for Process Group with ID {}, "
+                + "replicating request to re-enable {} affected services", groupId, affectedServices.size());
+
+            activateControllerServices(groupId, originalUri, jerseyClient, updateRequest, pause,
+                affectedServices, ControllerServiceState.ENABLED, updateRequest.getEnableServicesStep());
+        }
+
+        // restart processors
+        if (affectedProcessors != null) {
+            logger.info("In order to update Variable Registry for Process Group with ID {}, "
+                + "replicating request to restart {} affected processors", groupId, affectedProcessors.size());
+
+            scheduleProcessors(groupId, originalUri, jerseyClient, updateRequest, pause,
+                affectedProcessors, ScheduledState.RUNNING, updateRequest.getStartProcessorsStep());
+        }
+
+        updateRequest.setComplete(true);
+    }
+
+    /**
+     * Periodically polls the process group with the given ID, waiting for all processors whose ID's are given to have the given Scheduled State.
+     *
+     * @param client the Jersey Client to use for making the request
+     * @param groupId the ID of the Process Group to poll
+     * @param processorIds the ID of all Processors whose state should be equal to the given desired state
+     * @param desiredState the desired state for all processors with the ID's given
+     * @param pause the Pause that can be used to wait between polling
+     * @return <code>true</code> if successful, <code>false</code> if unable to wait for processors to reach the desired state
+     */
+    private boolean waitForProcessorStatus(final Client client, final URI originalUri, final String groupId, final Set<String> processorIds, final ScheduledState desiredState, final Pause pause) {
+        URI groupUri;
+        try {
+            groupUri = new URI(originalUri.getScheme(), originalUri.getUserInfo(), originalUri.getHost(),
+                originalUri.getPort(), "/nifi-api/flow/process-groups/" + groupId + "/status", "recursive=true", originalUri.getFragment());
+        } catch (URISyntaxException e) {
+            throw new RuntimeException(e);
+        }
+
+        boolean continuePolling = true;
+        while (continuePolling) {
+            final ClientResponse response = client.resource(groupUri).header("Content-Type", "application/json").get(ClientResponse.class);
+            if (response.getStatus() != Status.OK.getStatusCode()) {
+                return false;
+            }
+
+            final ProcessGroupStatusEntity statusEntity = response.getEntity(ProcessGroupStatusEntity.class);
+            final ProcessGroupStatusDTO statusDto = statusEntity.getProcessGroupStatus();
+            final ProcessGroupStatusSnapshotDTO statusSnapshotDto = statusDto.getAggregateSnapshot();
+
+            if (isProcessorStatusEqual(statusSnapshotDto, processorIds, desiredState)) {
+                logger.debug("All {} processors of interest now have the desired state of {}", processorIds.size(), desiredState);
+                return true;
+            }
+
+            // Not all of the processors are in the desired state. Pause for a bit and poll again.
+            continuePolling = pause.pause();
+        }
+
+        return false;
+    }
+
+    /**
+     * Periodically polls the process group with the given ID, waiting for all processors whose ID's are given to have the given Scheduled State.
+     *
+     * @param groupId the ID of the Process Group to poll
+     * @param processorIds the ID of all Processors whose state should be equal to the given desired state
+     * @param desiredState the desired state for all processors with the ID's given
+     * @param pause the Pause that can be used to wait between polling
+     * @return <code>true</code> if successful, <code>false</code> if unable to wait for processors to reach the desired state
+     */
+    private boolean waitForLocalProcessorStatus(final String groupId, final Set<String> processorIds, final ScheduledState desiredState, final Pause pause) {
+        boolean continuePolling = true;
+        while (continuePolling) {
+            final ProcessGroupStatusEntity statusEntity = serviceFacade.getProcessGroupStatus(groupId, true);
+            final ProcessGroupStatusDTO statusDto = statusEntity.getProcessGroupStatus();
+            final ProcessGroupStatusSnapshotDTO statusSnapshotDto = statusDto.getAggregateSnapshot();
+
+            if (isProcessorStatusEqual(statusSnapshotDto, processorIds, desiredState)) {
+                logger.debug("All {} processors of interest now have the desired state of {}", processorIds.size(), desiredState);
+                return true;
+            }
+
+            // Not all of the processors are in the desired state. Pause for a bit and poll again.
+            continuePolling = pause.pause();
+        }
+
+        return false;
+    }
+
+    private boolean isProcessorStatusEqual(final ProcessGroupStatusSnapshotDTO statusSnapshot, final Set<String> processorIds, final ScheduledState desiredState) {
+        final String desiredStateName = desiredState.name();
+
+        final boolean allProcessorsMatch = statusSnapshot.getProcessorStatusSnapshots().stream()
+            .map(entity -> entity.getProcessorStatusSnapshot())
+            .filter(status -> processorIds.contains(status.getId()))
+            .allMatch(status -> {
+                final String runStatus = status.getRunStatus();
+                final boolean stateMatches = desiredStateName.equalsIgnoreCase(runStatus);
+                if (!stateMatches) {
+                    return false;
+                }
+
+                if (desiredState == ScheduledState.STOPPED && status.getActiveThreadCount() != 0) {
+                    return false;
+                }
+
+                return true;
+            });
+
+        if (!allProcessorsMatch) {
+            return false;
+        }
+
+        for (final ProcessGroupStatusSnapshotEntity childGroupEntity : statusSnapshot.getProcessGroupStatusSnapshots()) {
+            final ProcessGroupStatusSnapshotDTO childGroupStatus = childGroupEntity.getProcessGroupStatusSnapshot();
+            final boolean allMatchChildLevel = isProcessorStatusEqual(childGroupStatus, processorIds, desiredState);
+            if (!allMatchChildLevel) {
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+
+
+    /**
+     * Periodically polls the process group with the given ID, waiting for all controller services whose ID's are given to have the given Controller Service State.
+     *
+     * @param client the Jersey Client to use for making the HTTP Request
+     * @param groupId the ID of the Process Group to poll
+     * @param serviceIds the ID of all Controller Services whose state should be equal to the given desired state
+     * @param desiredState the desired state for all services with the ID's given
+     * @param pause the Pause that can be used to wait between polling
+     * @return <code>true</code> if successful, <code>false</code> if unable to wait for services to reach the desired state
+     */
+    private boolean waitForControllerServiceStatus(final Client client, final URI originalUri, final String groupId, final Set<String> serviceIds, final ControllerServiceState desiredState,
+        final Pause pause) {
+        URI groupUri;
+        try {
+            groupUri = new URI(originalUri.getScheme(), originalUri.getUserInfo(), originalUri.getHost(),
+                originalUri.getPort(), "/nifi-api/flow/process-groups/" + groupId + "/controller-services", "includeAncestorGroups=false,includeDescendantGroups=true", originalUri.getFragment());
+        } catch (URISyntaxException e) {
+            throw new RuntimeException(e);
+        }
+
+        boolean continuePolling = true;
+        while (continuePolling) {
+            final ClientResponse response = client.resource(groupUri).header("Content-Type", "application/json").get(ClientResponse.class);
+            if (response.getStatus() != Status.OK.getStatusCode()) {
+                return false;
+            }
+
+            final ControllerServicesEntity controllerServicesEntity = response.getEntity(ControllerServicesEntity.class);
+            final Set<ControllerServiceEntity> serviceEntities = controllerServicesEntity.getControllerServices();
+
+            final String desiredStateName = desiredState.name();
+            final boolean allServicesMatch = serviceEntities.stream()
+                .map(entity -> entity.getComponent())
+                .filter(service -> serviceIds.contains(service.getId()))
+                .map(service -> service.getState())
+                .allMatch(state -> state.equals(desiredStateName));
+
+            if (allServicesMatch) {
+                logger.debug("All {} controller services of interest now have the desired state of {}", serviceIds.size(), desiredState);
+                return true;
+            }
+
+            // Not all of the processors are in the desired state. Pause for a bit and poll again.
+            continuePolling = pause.pause();
+        }
+
+        return false;
+    }
+
+
+    /**
+     * Periodically polls the process group with the given ID, waiting for all controller services whose ID's are given to have the given Controller Service State.
+     *
+     * @param groupId the ID of the Process Group to poll
+     * @param serviceIds the ID of all Controller Services whose state should be equal to the given desired state
+     * @param desiredState the desired state for all services with the ID's given
+     * @param pause the Pause that can be used to wait between polling
+     * @param user the user that is retrieving the controller services
+     * @return <code>true</code> if successful, <code>false</code> if unable to wait for services to reach the desired state
+     */
+    private boolean waitForLocalControllerServiceStatus(final String groupId, final Set<String> serviceIds, final ControllerServiceState desiredState, final Pause pause, final NiFiUser user) {
+        boolean continuePolling = true;
+        while (continuePolling) {
+            final Set<ControllerServiceEntity> serviceEntities = serviceFacade.getControllerServices(groupId, false, true, user);
+
+            final String desiredStateName = desiredState.name();
+            final boolean allServicesMatch = serviceEntities.stream()
+                .map(entity -> entity.getComponent())
+                .filter(service -> serviceIds.contains(service.getId()))
+                .map(service -> service.getState())
+                .allMatch(state -> desiredStateName.equals(state));
+
+            if (allServicesMatch) {
+                logger.debug("All {} controller services of interest now have the desired state of {}", serviceIds.size(), desiredState);
+                return true;
+            }
+
+            // Not all of the processors are in the desired state. Pause for a bit and poll again.
+            continuePolling = pause.pause();
+        }
+
+        return false;
+    }
+
+    private VariableRegistryUpdateRequest createVariableRegistryUpdateRequest(final String groupId) {
+        final VariableRegistryUpdateRequest updateRequest = new VariableRegistryUpdateRequest(UUID.randomUUID().toString(), groupId);
+
+        // before adding to the request map, purge any old requests. Must do this by creating a List of ID's
+        // and then removing those ID's one-at-a-time in order to avoid ConcurrentModificationException.
+        final Date oneMinuteAgo = new Date(System.currentTimeMillis() - VARIABLE_REGISTRY_UPDATE_REQUEST_EXPIRATION);
+        final List<String> completedRequestIds = varRegistryUpdateRequests.entrySet().stream()
+            .filter(entry -> entry.getValue().isComplete())
+            .filter(entry -> entry.getValue().getLastUpdated().before(oneMinuteAgo))
+            .map(Map.Entry::getKey)
+            .collect(Collectors.toList());
+
+        completedRequestIds.stream().forEach(id -> varRegistryUpdateRequests.remove(id));
+
+        final int requestCount = varRegistryUpdateRequests.size();
+        if (requestCount > MAX_VARIABLE_REGISTRY_UPDATE_REQUESTS) {
+            throw new IllegalStateException("There are already " + requestCount + " update requests for variable registries. "
+                + "Cannot issue any more requests until the older ones are deleted or expire");
+        }
+
+        this.varRegistryUpdateRequests.put(updateRequest.getRequestId(), updateRequest);
+        return updateRequest;
+    }
+
+    private Response updateVariableRegistryLocal(final String groupId, final List<AffectedComponentDTO> affectedProcessors, final List<AffectedComponentDTO> affectedServices,
+        final VariableRegistryEntity requestEntity) {
+
+        final Set<String> affectedProcessorIds = affectedProcessors == null ? Collections.emptySet() : affectedProcessors.stream()
+            .map(component -> component.getComponentId())
+            .collect(Collectors.toSet());
+        Map<String, Revision> processorRevisionMap = getRevisions(groupId, affectedProcessorIds);
+
+        final Set<String> affectedServiceIds = affectedServices == null ? Collections.emptySet() : affectedServices.stream()
+            .map(component -> component.getComponentId())
+            .collect(Collectors.toSet());
+        Map<String, Revision> serviceRevisionMap = getRevisions(groupId, affectedServiceIds);
+
+        // update the variable registry
+        final VariableRegistryUpdateRequest updateRequest = createVariableRegistryUpdateRequest(groupId);
+        updateRequest.getIdentifyRelevantComponentsStep().setComplete(true);
+        final Pause pause = createPause(updateRequest);
+
+        final Revision requestRevision = getRevision(requestEntity.getProcessGroupRevision(), groupId);
+
+        final NiFiUser user = NiFiUserUtils.getNiFiUser();
+        final Runnable updateTask = new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    // Stop processors
+                    performUpdateVariableRegistryStep(groupId, updateRequest, updateRequest.getStopProcessorsStep(), "Stopping Processors",
+                        () -> stopProcessors(user, updateRequest, groupId, processorRevisionMap, pause));
+
+                    // Update revision map because this will have modified the revisions of our components.
+                    final Map<String, Revision> updatedProcessorRevisionMap = getRevisions(groupId, affectedProcessorIds);
+
+                    // Disable controller services
+                    performUpdateVariableRegistryStep(groupId, updateRequest, updateRequest.getDisableServicesStep(), "Disabling Controller Services",
+                        () -> disableControllerServices(user, updateRequest, groupId, serviceRevisionMap, pause));
+
+                    // Update revision map because this will have modified the revisions of our components.
+                    final Map<String, Revision> updatedServiceRevisionMap = getRevisions(groupId, affectedServiceIds);
+
+                    // Apply the updates
+                    performUpdateVariableRegistryStep(groupId, updateRequest, updateRequest.getApplyUpdatesStep(), "Applying updates to Variable Registry",
+                        () -> serviceFacade.updateVariableRegistry(user, requestRevision, requestEntity.getVariableRegistry()));
+
+                    // Re-enable the controller services
+                    performUpdateVariableRegistryStep(groupId, updateRequest, updateRequest.getEnableServicesStep(), "Re-enabling Controller Services",
+                        () -> enableControllerServices(user, groupId, updatedServiceRevisionMap, pause));
+
+                    // Restart processors
+                    performUpdateVariableRegistryStep(groupId, updateRequest, updateRequest.getStartProcessorsStep(), "Restarting Processors",
+                        () -> startProcessors(user, groupId, updatedProcessorRevisionMap, pause));
+
+                    // Set complete
+                    updateRequest.setComplete(true);
+                    updateRequest.setLastUpdated(new Date());
+                } catch (final Exception e) {
+                    logger.error("Failed to update Variable Registry for Proces Group with ID " + groupId, e);
+                    updateRequest.setFailureReason("An unexpected error has occurred: " + e);
+                }
+            }
+        };
+
+        // Submit the task to be run in the background
+        variableRegistryThreadPool.submit(updateTask);
+
+        final VariableRegistryUpdateRequestEntity responseEntity = new VariableRegistryUpdateRequestEntity();
+        responseEntity.setRequestDto(dtoFactory.createVariableRegistryUpdateRequestDto(updateRequest));
+        responseEntity.getRequestDto().setUri(generateResourceUri("process-groups", groupId, "variable-registry", "update-requests", updateRequest.getRequestId()));
+
+        final URI location = URI.create(responseEntity.getRequestDto().getUri());
+        return Response.status(Status.ACCEPTED).location(location).entity(responseEntity).build();
+    }
+
+    private Map<String, Revision> getRevisions(final String groupId, final Set<String> componentIds) {
+        final Set<Revision> processorRevisions = serviceFacade.getRevisionsFromGroup(groupId, group -> componentIds);
+        return processorRevisions.stream().collect(Collectors.toMap(revision -> revision.getComponentId(), Function.identity()));
+    }
+
+    private void performUpdateVariableRegistryStep(final String groupId, final VariableRegistryUpdateRequest request, final VariableRegistryUpdateStep step,
+        final String stepDescription, final Runnable action) {
+
+        if (request.isComplete()) {
+            logger.info("In updating Variable Registry for Process Group with ID {}"
+                + ", skipping the following step because the request has completed already: {}", groupId, stepDescription);
+            return;
+        }
+
+        try {
+            logger.info("In order to update Variable Registry for Process Group with ID {}, {}", groupId, stepDescription);
+
+            action.run();
+            step.setComplete(true);
+        } catch (final Exception e) {
+            request.setComplete(true);
+            logger.error("Failed to update variable registry for Process Group with ID {}", groupId, e);
+
+            step.setComplete(true);
+            step.setFailureReason(e.getMessage());
+            request.setFailureReason("Failed to update Variable Registry because failed while performing step: " + stepDescription);
+        }
+
+        request.setLastUpdated(new Date());
+    }
+
+    private void stopProcessors(final NiFiUser user, final VariableRegistryUpdateRequest updateRequest, final String processGroupId,
+        final Map<String, Revision> processorRevisions, final Pause pause) {
+
+        if (processorRevisions.isEmpty()) {
+            return;
+        }
+
+        serviceFacade.verifyScheduleComponents(processGroupId, ScheduledState.STOPPED, processorRevisions.keySet());
+        serviceFacade.scheduleComponents(user, processGroupId, ScheduledState.STOPPED, processorRevisions);
+        waitForLocalProcessorStatus(processGroupId, processorRevisions.keySet(), ScheduledState.STOPPED, pause);
+    }
+
+    private void startProcessors(final NiFiUser user, final String processGroupId, final Map<String, Revision> processorRevisions, final Pause pause) {
+        if (processorRevisions.isEmpty()) {
+            return;
+        }
+
+        serviceFacade.verifyScheduleComponents(processGroupId, ScheduledState.RUNNING, processorRevisions.keySet());
+        serviceFacade.scheduleComponents(user, processGroupId, ScheduledState.RUNNING, processorRevisions);
+        waitForLocalProcessorStatus(processGroupId, processorRevisions.keySet(), ScheduledState.RUNNING, pause);
+    }
+
+    private void disableControllerServices(final NiFiUser user, final VariableRegistryUpdateRequest updateRequest, final String processGroupId,
+        final Map<String, Revision> serviceRevisions, final Pause pause) {
+
+        if (serviceRevisions.isEmpty()) {
+            return;
+        }
+
+        serviceFacade.verifyActivateControllerServices(processGroupId, ControllerServiceState.DISABLED, serviceRevisions.keySet());
+        serviceFacade.activateControllerServices(user, processGroupId, ControllerServiceState.DISABLED, serviceRevisions);
+        waitForLocalControllerServiceStatus(processGroupId, serviceRevisions.keySet(), ControllerServiceState.DISABLED, pause, user);
+    }
+
+    private void enableControllerServices(final NiFiUser user, final String processGroupId, final Map<String, Revision> serviceRevisions, final Pause pause) {
+        if (serviceRevisions.isEmpty()) {
+            return;
+        }
+
+        serviceFacade.verifyActivateControllerServices(processGroupId, ControllerServiceState.ENABLED, serviceRevisions.keySet());
+        serviceFacade.activateControllerServices(user, processGroupId, ControllerServiceState.ENABLED, serviceRevisions);
+        waitForLocalControllerServiceStatus(processGroupId, serviceRevisions.keySet(), ControllerServiceState.ENABLED, pause, user);
+    }
+
+
+    private void scheduleProcessors(final String groupId, final URI originalUri, final Client jerseyClient, final VariableRegistryUpdateRequest updateRequest,
+        final Pause pause, final Collection<AffectedComponentDTO> affectedProcessors, final ScheduledState desiredState, final VariableRegistryUpdateStep updateStep) {
+        final Set<String> affectedProcessorIds = affectedProcessors.stream()
+            .map(component -> component.getComponentId())
+            .collect(Collectors.toSet());
+
+        final Map<String, Revision> processorRevisionMap = getRevisions(groupId, affectedProcessorIds);
+        final Map<String, RevisionDTO> processorRevisionDtoMap = processorRevisionMap.entrySet().stream().collect(
+            Collectors.toMap(Map.Entry::getKey, entry -> dtoFactory.createRevisionDTO(entry.getValue())));
+
+        final ScheduleComponentsEntity stopProcessorsEntity = new ScheduleComponentsEntity();
+        stopProcessorsEntity.setComponents(processorRevisionDtoMap);
+        stopProcessorsEntity.setId(groupId);
+        stopProcessorsEntity.setState(desiredState.name());
+
+        URI stopProcessorUri;
+        try {
+            stopProcessorUri = new URI(originalUri.getScheme(), originalUri.getUserInfo(), originalUri.getHost(),
+                originalUri.getPort(), "/nifi-api/flow/process-groups/" + groupId, null, originalUri.getFragment());
+        } catch (URISyntaxException e) {
+            throw new RuntimeException(e);
+        }
+
+        final ClientResponse stopProcessorResponse = jerseyClient.resource(stopProcessorUri)
+            .header("Content-Type", "application/json")
+            .entity(stopProcessorsEntity)
+            .put(ClientResponse.class);
+
+        final int stopProcessorStatus = stopProcessorResponse.getStatus();
+        if (stopProcessorStatus != Status.OK.getStatusCode()) {
+            updateRequest.getStopProcessorsStep().setFailureReason("Failed while " + updateStep.getDescription());
+            updateRequest.setFailureReason("Failed while " + updateStep.getDescription());
+            return;
+        }
+
+        updateRequest.setLastUpdated(new Date());
+        final boolean processorsTransitioned = waitForProcessorStatus(jerseyClient, originalUri, groupId, affectedProcessorIds, desiredState, pause);
+        if (processorsTransitioned) {
+            updateStep.setComplete(true);
+        } else {
+            updateStep.setFailureReason("Failed while " + updateStep.getDescription());
+            updateRequest.setFailureReason("Failed while " + updateStep.getDescription());
+            return;
+        }
+    }
+
+    private void activateControllerServices(final String groupId, final URI originalUri, final Client jerseyClient, final VariableRegistryUpdateRequest updateRequest,
+        final Pause pause, final Collection<AffectedComponentDTO> affectedServices, final ControllerServiceState desiredState, final VariableRegistryUpdateStep updateStep) {
+
+        final Set<String> affectedServiceIds = affectedServices.stream()
+            .map(component -> component.getComponentId())
+            .collect(Collectors.toSet());
+
+        final Map<String, Revision> serviceRevisionMap = getRevisions(groupId, affectedServiceIds);
+        final Map<String, RevisionDTO> serviceRevisionDtoMap = serviceRevisionMap.entrySet().stream().collect(
+            Collectors.toMap(Map.Entry::getKey, entry -> dtoFactory.createRevisionDTO(entry.getValue())));
+
+        final ActivateControllerServicesEntity disableServicesEntity = new ActivateControllerServicesEntity();
+        disableServicesEntity.setComponents(serviceRevisionDtoMap);
+        disableServicesEntity.setId(groupId);
+        disableServicesEntity.setState(desiredState.name());
+
+        URI disableServicesUri;
+        try {
+            disableServicesUri = new URI(originalUri.getScheme(), originalUri.getUserInfo(), originalUri.getHost(),
+                originalUri.getPort(), "/nifi-api/flow/process-groups/" + groupId + "/controller-services", null, originalUri.getFragment());
+        } catch (URISyntaxException e) {
+            throw new RuntimeException(e);
+        }
+
+        final ClientResponse disableServicesResponse = jerseyClient.resource(disableServicesUri)
+            .header("Content-Type", "application/json")
+            .entity(disableServicesEntity)
+            .put(ClientResponse.class);
+
+        final int disableServicesStatus = disableServicesResponse.getStatus();
+        if (disableServicesStatus != Status.OK.getStatusCode()) {
+            updateStep.setFailureReason("Failed while " + updateStep.getDescription());
+            updateRequest.setFailureReason("Failed while " + updateStep.getDescription());
+            return;
+        }
+
+        updateRequest.setLastUpdated(new Date());
+        if (waitForControllerServiceStatus(jerseyClient, originalUri, groupId, affectedServiceIds, desiredState, pause)) {
+            updateStep.setComplete(true);
+        } else {
+            updateStep.setFailureReason("Failed while " + updateStep.getDescription());
+            updateRequest.setFailureReason("Failed while " + updateStep.getDescription());
+            return;
+        }
+    }
+
+
+    private void applyVariableRegistryUpdate(final String groupId, final URI originalUri, final Client jerseyClient, final VariableRegistryUpdateRequest updateRequest,
+        final VariableRegistryEntity updateEntity) {
+
+        URI applyUpdatesUri;
+        try {
+            applyUpdatesUri = new URI(originalUri.getScheme(), originalUri.getUserInfo(), originalUri.getHost(),
+                originalUri.getPort(), "/nifi-api/process-groups/" + groupId + "/variable-registry", null, originalUri.getFragment());
+        } catch (URISyntaxException e) {
+            throw new RuntimeException(e);
+        }
+
+        final ClientResponse applyUpdatesResponse = jerseyClient.resource(applyUpdatesUri)
+            .header("Content-Type", "application/json")
+            .entity(updateEntity)
+            .put(ClientResponse.class);
+
+        final int applyUpdatesStatus = applyUpdatesResponse.getStatus();
+        updateRequest.setLastUpdated(new Date());
+        if (applyUpdatesStatus != Status.OK.getStatusCode()) {
+            updateRequest.getApplyUpdatesStep().setFailureReason("Failed to apply updates to the Variable Registry");
+            updateRequest.setFailureReason("Failed to apply updates to the Variable Registry");
+            return;
+        }
+    }
+
     /**
      * Removes the specified process group reference.
      *
@@ -2426,4 +3378,8 @@ public class ProcessGroupResource extends ApplicationResource {
     public void setAuthorizer(Authorizer authorizer) {
         this.authorizer = authorizer;
     }
+
+    public void setDtoFactory(DtoFactory dtoFactory) {
+        this.dtoFactory = dtoFactory;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
index 4332a0c..a445e49 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
@@ -16,6 +16,33 @@
  */
 package org.apache.nifi.web.api.dto;
 
+import java.text.Collator;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TimeZone;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import javax.ws.rs.WebApplicationException;
+
 import org.apache.commons.lang3.ClassUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.action.Action;
@@ -47,6 +74,7 @@ import org.apache.nifi.authorization.Resource;
 import org.apache.nifi.authorization.User;
 import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.resource.ComponentAuthorizable;
+import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.authorization.user.NiFiUserUtils;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.bundle.BundleCoordinate;
@@ -110,6 +138,9 @@ import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
 import org.apache.nifi.provenance.lineage.LineageEdge;
 import org.apache.nifi.provenance.lineage.LineageNode;
 import org.apache.nifi.provenance.lineage.ProvenanceEventLineageNode;
+import org.apache.nifi.registry.ComponentVariableRegistry;
+import org.apache.nifi.registry.variable.VariableRegistryUpdateRequest;
+import org.apache.nifi.registry.variable.VariableRegistryUpdateStep;
 import org.apache.nifi.remote.RemoteGroupPort;
 import org.apache.nifi.remote.RootGroupPort;
 import org.apache.nifi.reporting.Bulletin;
@@ -161,35 +192,10 @@ import org.apache.nifi.web.api.entity.ProcessGroupStatusSnapshotEntity;
 import org.apache.nifi.web.api.entity.ProcessorStatusSnapshotEntity;
 import org.apache.nifi.web.api.entity.RemoteProcessGroupStatusSnapshotEntity;
 import org.apache.nifi.web.api.entity.TenantEntity;
+import org.apache.nifi.web.api.entity.VariableEntity;
 import org.apache.nifi.web.controller.ControllerFacade;
 import org.apache.nifi.web.revision.RevisionManager;
 
-import javax.ws.rs.WebApplicationException;
-import java.text.Collator;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TimeZone;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
-import java.util.function.Function;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
 public final class DtoFactory {
 
     @SuppressWarnings("rawtypes")
@@ -1712,9 +1718,34 @@ public final class DtoFactory {
      * @return dto
      */
     public PermissionsDTO createPermissionsDto(final Authorizable authorizable) {
+        return createPermissionsDto(authorizable, NiFiUserUtils.getNiFiUser());
+    }
+
+    /**
+     * Creates the PermissionsDTO based on the specified Authorizable for the given user
+     *
+     * @param authorizable authorizable
+     * @param user the NiFi User for which the Permissions are being created
+     * @return dto
+     */
+    public PermissionsDTO createPermissionsDto(final Authorizable authorizable, final NiFiUser user) {
         final PermissionsDTO dto = new PermissionsDTO();
-        dto.setCanRead(authorizable.isAuthorized(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser()));
-        dto.setCanWrite(authorizable.isAuthorized(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()));
+        dto.setCanRead(authorizable.isAuthorized(authorizer, RequestAction.READ, user));
+        dto.setCanWrite(authorizable.isAuthorized(authorizer, RequestAction.WRITE, user));
+        return dto;
+    }
+
+    public AffectedComponentDTO createAffectedComponentDto(final ConfiguredComponent component) {
+        final AffectedComponentDTO dto = new AffectedComponentDTO();
+        dto.setComponentId(component.getIdentifier());
+        dto.setParentGroupId(component.getProcessGroupIdentifier());
+
+        if (component instanceof ProcessorNode) {
+            dto.setComponentType(AffectedComponentDTO.COMPONENT_TYPE_PROCESSOR);
+        } else if (component instanceof ControllerServiceNode) {
+            dto.setComponentType(AffectedComponentDTO.COMPONENT_TYPE_CONTROLLER_SERVICE);
+        }
+
         return dto;
     }
 
@@ -1998,6 +2029,10 @@ public final class DtoFactory {
         dto.setComments(group.getComments());
         dto.setName(group.getName());
 
+        final Map<String, String> variables = group.getVariableRegistry().getVariableMap().entrySet().stream()
+            .collect(Collectors.toMap(entry -> entry.getKey().getName(), entry -> entry.getValue()));
+        group.setVariables(variables);
+
         final ProcessGroup parentGroup = group.getParent();
         if (parentGroup != null) {
             dto.setParentGroupId(parentGroup.getIdentifier());
@@ -2079,6 +2114,128 @@ public final class DtoFactory {
         return deprecationNotice == null ? null : deprecationNotice.reason();
     }
 
+
+    public VariableRegistryDTO createVariableRegistryDto(final ProcessGroup processGroup) {
+        final ComponentVariableRegistry variableRegistry = processGroup.getVariableRegistry();
+
+        final List<String> variableNames = variableRegistry.getVariableMap().keySet().stream()
+            .map(descriptor -> descriptor.getName())
+            .collect(Collectors.toList());
+
+        final Set<VariableEntity> variableEntities = new LinkedHashSet<>();
+
+        for (final String variableName : variableNames) {
+            final VariableDTO variableDto = new VariableDTO();
+            variableDto.setName(variableName);
+            variableDto.setValue(variableRegistry.getVariableValue(variableName));
+            variableDto.setProcessGroupId(processGroup.getIdentifier());
+
+            final Set<ConfiguredComponent> affectedComponents = processGroup.getComponentsAffectedByVariable(variableName);
+            final Set<AffectedComponentDTO> affectedComponentDtos = affectedComponents.stream()
+                .map(component -> createAffectedComponentDto(component))
+                .collect(Collectors.toSet());
+
+            boolean canWrite = true;
+            for (final ConfiguredComponent component : affectedComponents) {
+                final PermissionsDTO permissions = createPermissionsDto(component);
+                if (!permissions.getCanRead() || !permissions.getCanWrite()) {
+                    canWrite = false;
+                    break;
+                }
+            }
+
+            variableDto.setAffectedComponents(affectedComponentDtos);
+
+            final VariableEntity variableEntity = new VariableEntity();
+            variableEntity.setVariable(variableDto);
+            variableEntity.setCanWrite(canWrite);
+
+            variableEntities.add(variableEntity);
+        }
+
+        final VariableRegistryDTO registryDto = new VariableRegistryDTO();
+        registryDto.setProcessGroupId(processGroup.getIdentifier());
+        registryDto.setVariables(variableEntities);
+
+        return registryDto;
+    }
+
+    public VariableRegistryUpdateRequestDTO createVariableRegistryUpdateRequestDto(final VariableRegistryUpdateRequest request) {
+        final VariableRegistryUpdateRequestDTO dto = new VariableRegistryUpdateRequestDTO();
+        dto.setComplete(request.isComplete());
+        dto.setFailureReason(request.getFailureReason());
+        dto.setLastUpdated(request.getLastUpdated());
+        dto.setProcessGroupId(request.getProcessGroupId());
+        dto.setRequestId(request.getRequestId());
+        dto.setSubmissionTime(request.getSubmissionTime());
+
+        final List<VariableRegistryUpdateStepDTO> updateSteps = new ArrayList<>();
+        updateSteps.add(createVariableRegistryUpdateStepDto(request.getIdentifyRelevantComponentsStep()));
+        updateSteps.add(createVariableRegistryUpdateStepDto(request.getStopProcessorsStep()));
+        updateSteps.add(createVariableRegistryUpdateStepDto(request.getDisableServicesStep()));
+        updateSteps.add(createVariableRegistryUpdateStepDto(request.getApplyUpdatesStep()));
+        updateSteps.add(createVariableRegistryUpdateStepDto(request.getEnableServicesStep()));
+        updateSteps.add(createVariableRegistryUpdateStepDto(request.getStartProcessorsStep()));
+        dto.setUpdateSteps(updateSteps);
+
+        return dto;
+    }
+
+    public VariableRegistryUpdateStepDTO createVariableRegistryUpdateStepDto(final VariableRegistryUpdateStep step) {
+        final VariableRegistryUpdateStepDTO dto = new VariableRegistryUpdateStepDTO();
+        dto.setComplete(step.isComplete());
+        dto.setDescription(step.getDescription());
+        dto.setFailureReason(step.getFailureReason());
+        return dto;
+    }
+
+
+    public VariableRegistryDTO populateAffectedComponents(final VariableRegistryDTO variableRegistry, final ProcessGroup group) {
+        if (!group.getIdentifier().equals(variableRegistry.getProcessGroupId())) {
+            throw new IllegalArgumentException("Variable Registry does not have the same Group ID as the given Process Group");
+        }
+
+        final Set<VariableEntity> variableEntities = new LinkedHashSet<>();
+
+        for (final VariableEntity inputEntity : variableRegistry.getVariables()) {
+            final VariableEntity entity = new VariableEntity();
+
+            final VariableDTO inputDto = inputEntity.getVariable();
+            final VariableDTO variableDto = new VariableDTO();
+            variableDto.setName(inputDto.getName());
+            variableDto.setValue(inputDto.getValue());
+            variableDto.setProcessGroupId(group.getIdentifier());
+
+            final Set<ConfiguredComponent> affectedComponents = group.getComponentsAffectedByVariable(variableDto.getName());
+            final Set<AffectedComponentDTO> affectedComponentDtos = affectedComponents.stream()
+                .map(component -> createAffectedComponentDto(component))
+                .collect(Collectors.toSet());
+
+            boolean canWrite = true;
+            for (final ConfiguredComponent component : affectedComponents) {
+                final PermissionsDTO permissions = createPermissionsDto(component);
+                if (!permissions.getCanRead() || !permissions.getCanWrite()) {
+                    canWrite = false;
+                    break;
+                }
+            }
+
+            variableDto.setAffectedComponents(affectedComponentDtos);
+
+            entity.setCanWrite(canWrite);
+            entity.setVariable(inputDto);
+
+            variableEntities.add(entity);
+        }
+
+        final VariableRegistryDTO registryDto = new VariableRegistryDTO();
+        registryDto.setProcessGroupId(group.getIdentifier());
+        registryDto.setVariables(variableEntities);
+
+        return registryDto;
+    }
+
+
     /**
      * Gets the capability description from the specified class.
      */
@@ -3016,6 +3173,10 @@ public final class DtoFactory {
         copy.setActiveRemotePortCount(original.getActiveRemotePortCount());
         copy.setInactiveRemotePortCount(original.getInactiveRemotePortCount());
 
+        if (original.getVariables() != null) {
+            copy.setVariables(new HashMap<>(original.getVariables()));
+        }
+
         return copy;
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java
index 41249ba..a7f370a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java
@@ -65,6 +65,7 @@ import org.apache.nifi.web.api.entity.StatusHistoryEntity;
 import org.apache.nifi.web.api.entity.TenantEntity;
 import org.apache.nifi.web.api.entity.UserEntity;
 import org.apache.nifi.web.api.entity.UserGroupEntity;
+import org.apache.nifi.web.api.entity.VariableRegistryEntity;
 
 import java.util.Date;
 import java.util.List;
@@ -431,6 +432,18 @@ public final class EntityFactory {
         return entity;
     }
 
+    public VariableRegistryEntity createVariableRegistryEntity(final VariableRegistryDTO dto, final RevisionDTO revision, final PermissionsDTO permissions) {
+        final VariableRegistryEntity entity = new VariableRegistryEntity();
+        entity.setProcessGroupRevision(revision);
+        if (dto != null) {
+            if (permissions != null && permissions.getCanRead()) {
+                entity.setVariableRegistry(dto);
+            }
+        }
+
+        return entity;
+    }
+
     public ControllerServiceEntity createControllerServiceEntity(final ControllerServiceDTO dto, final RevisionDTO revision, final PermissionsDTO permissions, final List<BulletinEntity> bulletins) {
         final ControllerServiceEntity entity = new ControllerServiceEntity();
         entity.setRevision(revision);

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java
index fa92425..0409e95 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java
@@ -63,7 +63,7 @@ public interface ControllerServiceDAO {
      *
      * @return The controller services
      */
-    Set<ControllerServiceNode> getControllerServices(String groupId);
+    Set<ControllerServiceNode> getControllerServices(String groupId, boolean includeAncestorGroups, boolean includeDescendantGroups);
 
     /**
      * Updates the specified controller service.

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessGroupDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessGroupDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessGroupDAO.java
index 155b36e..d7ca806 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessGroupDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessGroupDAO.java
@@ -16,11 +16,14 @@
  */
 package org.apache.nifi.web.dao;
 
+import java.util.Set;
+import java.util.concurrent.Future;
+
 import org.apache.nifi.controller.ScheduledState;
+import org.apache.nifi.controller.service.ControllerServiceState;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.web.api.dto.ProcessGroupDTO;
-
-import java.util.Set;
+import org.apache.nifi.web.api.dto.VariableRegistryDTO;
 
 public interface ProcessGroupDAO {
 
@@ -65,12 +68,32 @@ public interface ProcessGroupDAO {
     void verifyScheduleComponents(String groupId, ScheduledState state, Set<String> componentIds);
 
     /**
+     * Verifies the specified controller services can be modified
+     *
+     * @param groupId the ID of the process group
+     * @param state the desired state
+     * @param serviceIds the ID's of the controller services
+     */
+    void verifyActivateControllerServices(String groupId, ControllerServiceState state, Set<String> serviceIds);
+
+    /**
      * Schedules the components in the specified process group.
      *
      * @param groupId id
      * @param state scheduled state
+     *
+     * @return a Future that can be used to wait for the services to finish starting or stopping
      */
-    void scheduleComponents(String groupId, ScheduledState state, Set<String> componentIds);
+    Future<Void> scheduleComponents(String groupId, ScheduledState state, Set<String> componentIds);
+
+    /**
+     * Enables or disables the controller services in the specified process group
+     *
+     * @param groupId the id of the group
+     * @param state the desired state
+     * @param serviceIds the ID's of the services to enable or disable
+     */
+    Future<Void> activateControllerServices(String groupId, ControllerServiceState state, Set<String> serviceIds);
 
     /**
      * Updates the specified process group.
@@ -81,6 +104,21 @@ public interface ProcessGroupDAO {
     ProcessGroup updateProcessGroup(ProcessGroupDTO processGroup);
 
     /**
+     * Updates the specified variable registry
+     *
+     * @param variableRegistry the Variable Registry
+     * @return the Process Group that was updated
+     */
+    ProcessGroup updateVariableRegistry(VariableRegistryDTO variableRegistry);
+
+    /**
+     * Verifies that the specified updates to a current Process Group can be applied at this time
+     *
+     * @param processGroup the DTO That describes the changes to occur
+     */
+    void verifyUpdate(ProcessGroupDTO processGroup);
+
+    /**
      * Verifies the specified process group can be removed.
      *
      * @param groupId id

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java
index 36cf85b..0f9ec7a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java
@@ -124,7 +124,7 @@ public class StandardControllerServiceDAO extends ComponentDAO implements Contro
     }
 
     @Override
-    public Set<ControllerServiceNode> getControllerServices(final String groupId) {
+    public Set<ControllerServiceNode> getControllerServices(final String groupId, final boolean includeAncestorGroups, final boolean includeDescendantGroups) {
         if (groupId == null) {
             return flowController.getRootControllerServices();
         } else {
@@ -134,7 +134,12 @@ public class StandardControllerServiceDAO extends ComponentDAO implements Contro
                 throw new ResourceNotFoundException("Could not find Process Group with ID " + groupId);
             }
 
-            return procGroup.getControllerServices(true);
+            final Set<ControllerServiceNode> serviceNodes = procGroup.getControllerServices(includeAncestorGroups);
+            if (includeDescendantGroups) {
+                serviceNodes.addAll(procGroup.findAllControllerServices());
+            }
+
+            return serviceNodes;
         }
     }
 


[8/9] nifi git commit: NIFI-4224: - Initial implementation of Process Group level Variable Registry - Updated to incorporate PR Feedback - Changed log message because slf4j-simple apparently has a memory leak; passing a String instead of passing in the C

Posted by mc...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java
index 4b1ce59..fe2c060 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java
@@ -16,209 +16,21 @@
  */
 package org.apache.nifi.attribute.expression.language;
 
-import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionLexer;
-import org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser;
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.DateEvaluator;
+import org.antlr.runtime.tree.Tree;
+import org.apache.nifi.attribute.expression.language.compile.CompiledExpression;
+import org.apache.nifi.attribute.expression.language.compile.ExpressionCompiler;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.cast.BooleanCastEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.cast.DateCastEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.cast.DecimalCastEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.cast.NumberCastEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.cast.StringCastEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.cast.WholeNumberCastEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.AndEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.AppendEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.AttributeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ContainsEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.DivideEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.EndsWithEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.EqualsEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.EqualsIgnoreCaseEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.CharSequenceTranslatorEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.FindEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.FormatEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.FromRadixEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.GetDelimitedFieldEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.GetStateVariableEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.GreaterThanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.GreaterThanOrEqualEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.HostnameEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.IPEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.IfElseEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.InEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.IndexOfEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.IsEmptyEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.IsNullEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.JsonPathEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.LastIndexOfEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.LengthEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.LessThanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.LessThanOrEqualEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.MatchesEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.MathEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.MinusEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ModEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.MultiplyEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.NotEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.NotNullEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.NowEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.NumberToDateEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.OneUpSequenceEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.OrEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.PlusEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.PrependEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.RandomNumberGeneratorEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceAllEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceEmptyEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceFirstEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceNullEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.StartsWithEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.StringToDateEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringAfterEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringAfterLastEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringBeforeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringBeforeLastEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ToLowerEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ToRadixEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ToStringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ToUpperEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.TrimEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.UrlDecodeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.UrlEncodeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.Base64DecodeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.Base64EncodeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.UuidEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.literals.BooleanLiteralEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.literals.DecimalLiteralEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.literals.StringLiteralEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.literals.ToLiteralEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.literals.WholeNumberLiteralEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.reduce.CountEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.reduce.JoinEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.reduce.ReduceEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.selection.AllAttributesEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.selection.AnyAttributeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.selection.DelineatedAttributeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.selection.IteratingEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiAttributeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiMatchAttributeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiNamedAttributeEvaluator;
-import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageException;
 import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
 import org.apache.nifi.expression.AttributeValueDecorator;
-import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.exception.ProcessException;
-import org.antlr.runtime.ANTLRStringStream;
-import org.antlr.runtime.CharStream;
-import org.antlr.runtime.CommonTokenStream;
-import org.antlr.runtime.tree.Tree;
-
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.FROM_RADIX;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IF_ELSE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MATH;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ALL_ATTRIBUTES;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ALL_DELINEATED_VALUES;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ALL_MATCHING_ATTRIBUTES;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.AND;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ANY_ATTRIBUTE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ANY_DELINEATED_VALUE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ANY_MATCHING_ATTRIBUTE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.APPEND;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ATTRIBUTE_REFERENCE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ATTR_NAME;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.CONTAINS;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.DECIMAL;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IN;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.COUNT;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.DIVIDE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ENDS_WITH;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.EQUALS;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.EQUALS_IGNORE_CASE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.EXPRESSION;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.FALSE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.FIND;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.FORMAT;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.GET_DELIMITED_FIELD;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.GET_STATE_VALUE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.GREATER_THAN;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.GREATER_THAN_OR_EQUAL;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.HOSTNAME;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.INDEX_OF;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IP;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IS_EMPTY;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IS_NULL;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.JOIN;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.JSON_PATH;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LAST_INDEX_OF;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LENGTH;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LESS_THAN;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LESS_THAN_OR_EQUAL;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MATCHES;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MINUS;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MOD;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MULTIPLY;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MULTI_ATTRIBUTE_REFERENCE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NEXT_INT;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NOT;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NOT_NULL;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NOW;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_DECIMAL;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.WHOLE_NUMBER;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.OR;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.PLUS;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.PREPEND;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.RANDOM;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE_ALL;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE_EMPTY;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE_FIRST;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE_NULL;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.STARTS_WITH;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.STRING_LITERAL;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_AFTER;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_AFTER_LAST;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_BEFORE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_BEFORE_LAST;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_DATE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_LITERAL;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_LOWER;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_NUMBER;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_RADIX;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_STRING;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_UPPER;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TRIM;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TRUE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.URL_DECODE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.URL_ENCODE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.BASE64_DECODE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.BASE64_ENCODE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ESCAPE_JSON;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ESCAPE_CSV;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ESCAPE_HTML3;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ESCAPE_HTML4;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ESCAPE_XML;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UNESCAPE_JSON;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UNESCAPE_CSV;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UNESCAPE_HTML3;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UNESCAPE_HTML4;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UNESCAPE_XML;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UUID;
-
-import org.apache.nifi.attribute.expression.language.evaluation.selection.MappingEvaluator;
 
 /**
  * Class used for creating and evaluating NiFi Expression Language. Once a Query
@@ -413,18 +225,6 @@ public class Query {
         return Query.prepare(rawValue).evaluateExpressions(valueLookup, decorator);
     }
 
-    private static Evaluator<?> getRootSubjectEvaluator(final Evaluator<?> evaluator) {
-        if (evaluator == null) {
-            return null;
-        }
-
-        final Evaluator<?> subject = evaluator.getSubjectEvaluator();
-        if (subject == null) {
-            return evaluator;
-        }
-
-        return getRootSubjectEvaluator(subject);
-    }
 
     /**
      * Un-escapes ${...} patterns that were escaped
@@ -436,28 +236,11 @@ public class Query {
         return value.replaceAll("\\$\\$(?=\\$*\\{.*?\\})", "\\$");
     }
 
-
     public static Query fromTree(final Tree tree, final String text) {
-        return new Query(text, tree, buildEvaluator(tree));
+        final ExpressionCompiler compiler = new ExpressionCompiler();
+        return new Query(text, tree, compiler.buildEvaluator(tree));
     }
 
-    public static Tree compileTree(final String query) throws AttributeExpressionLanguageParsingException {
-        try {
-            final CommonTokenStream lexerTokenStream = createTokenStream(query);
-            final AttributeExpressionParser parser = new AttributeExpressionParser(lexerTokenStream);
-            final Tree ast = (Tree) parser.query().getTree();
-            final Tree tree = ast.getChild(0);
-
-            // ensure that we are able to build the evaluators, so that we validate syntax
-            final Evaluator<?> evaluator = buildEvaluator(tree);
-            verifyMappingEvaluatorReduced(evaluator);
-            return tree;
-        } catch (final AttributeExpressionLanguageParsingException e) {
-            throw e;
-        } catch (final Exception e) {
-            throw new AttributeExpressionLanguageParsingException(e);
-        }
-    }
 
     public static PreparedQuery prepare(final String query) throws AttributeExpressionLanguageParsingException {
         if (query == null) {
@@ -470,9 +253,11 @@ public class Query {
             return new EmptyPreparedQuery(query.replace("$$", "$"));
         }
 
+        final ExpressionCompiler compiler = new ExpressionCompiler();
+
         try {
             final List<String> substrings = new ArrayList<>();
-            final Map<String, Tree> trees = new HashMap<>();
+            final Map<String, CompiledExpression> compiledExpressions = new HashMap<>();
 
             int lastIndex = 0;
             for (final Range range : ranges) {
@@ -483,7 +268,10 @@ public class Query {
 
                 final String treeText = query.substring(range.getStart(), range.getEnd() + 1).replace("$$", "$");
                 substrings.add(treeText);
-                trees.put(treeText, Query.compileTree(treeText));
+
+                final CompiledExpression compiledExpression = compiler.compile(treeText);
+
+                compiledExpressions.put(treeText, compiledExpression);
                 lastIndex = range.getEnd() + 1;
             }
 
@@ -493,7 +281,7 @@ public class Query {
                 substrings.add(treeText);
             }
 
-            return new StandardPreparedQuery(substrings, trees);
+            return new StandardPreparedQuery(substrings, compiledExpressions);
         } catch (final AttributeExpressionLanguageParsingException e) {
             return new InvalidPreparedQuery(query, e.getMessage());
         }
@@ -501,15 +289,10 @@ public class Query {
 
     public static Query compile(final String query) throws AttributeExpressionLanguageParsingException {
         try {
-            final CommonTokenStream lexerTokenStream = createTokenStream(query);
-            final AttributeExpressionParser parser = new AttributeExpressionParser(lexerTokenStream);
-            final Tree ast = (Tree) parser.query().getTree();
-            final Tree tree = ast.getChild(0);
+            final ExpressionCompiler compiler = new ExpressionCompiler();
+            final CompiledExpression compiledExpression = compiler.compile(query);
 
-            final Evaluator<?> evaluator = buildEvaluator(tree);
-            verifyMappingEvaluatorReduced(evaluator);
-
-            return new Query(query, tree, evaluator);
+            return new Query(compiledExpression.getExpression(), compiledExpression.getTree(), compiledExpression.getRootEvaluator());
         } catch (final AttributeExpressionLanguageParsingException e) {
             throw e;
         } catch (final Exception e) {
@@ -517,57 +300,6 @@ public class Query {
         }
     }
 
-    private static void verifyMappingEvaluatorReduced(final Evaluator<?> evaluator) {
-        final Evaluator<?> rightMostEvaluator;
-        if (evaluator instanceof IteratingEvaluator) {
-            rightMostEvaluator = ((IteratingEvaluator<?>) evaluator).getLogicEvaluator();
-        } else {
-            rightMostEvaluator = evaluator;
-        }
-
-        Evaluator<?> eval = rightMostEvaluator.getSubjectEvaluator();
-        Evaluator<?> lastEval = rightMostEvaluator;
-        while (eval != null) {
-            if (eval instanceof ReduceEvaluator) {
-                throw new AttributeExpressionLanguageParsingException("Expression attempts to call function '" + lastEval.getToken() + "' on the result of '" + eval.getToken() +
-                    "'. This is not allowed. Instead, use \"${literal( ${<embedded expression>} ):" + lastEval.getToken() + "(...)}\"");
-            }
-
-            lastEval = eval;
-            eval = eval.getSubjectEvaluator();
-        }
-
-        // if the result type of the evaluator is BOOLEAN, then it will always
-        // be reduced when evaluator.
-        final ResultType resultType = evaluator.getResultType();
-        if (resultType == ResultType.BOOLEAN) {
-            return;
-        }
-
-        final Evaluator<?> rootEvaluator = getRootSubjectEvaluator(evaluator);
-        if (rootEvaluator != null && rootEvaluator instanceof MultiAttributeEvaluator) {
-            final MultiAttributeEvaluator multiAttrEval = (MultiAttributeEvaluator) rootEvaluator;
-            switch (multiAttrEval.getEvaluationType()) {
-                case ALL_ATTRIBUTES:
-                case ALL_MATCHING_ATTRIBUTES:
-                case ALL_DELINEATED_VALUES: {
-                    if (!(evaluator instanceof ReduceEvaluator)) {
-                        throw new AttributeExpressionLanguageParsingException("Cannot evaluate expression because it attempts to reference multiple attributes but does not use a reducing function");
-                    }
-                    break;
-                }
-                default:
-                    throw new AttributeExpressionLanguageParsingException("Cannot evaluate expression because it attempts to reference multiple attributes but does not use a reducing function");
-            }
-        }
-    }
-
-    private static CommonTokenStream createTokenStream(final String expression) throws AttributeExpressionLanguageParsingException {
-        final CharStream input = new ANTLRStringStream(expression);
-        final AttributeExpressionLexer lexer = new AttributeExpressionLexer(input);
-        return new CommonTokenStream(lexer);
-    }
-
     public ResultType getResultType() {
         return evaluator.getResultType();
     }
@@ -598,789 +330,7 @@ public class Query {
         return "Query [" + query + "]";
     }
 
-    private static Evaluator<String> newStringLiteralEvaluator(final String literalValue) {
-        if (literalValue == null || literalValue.length() < 2) {
-            return new StringLiteralEvaluator(literalValue);
-        }
-
-        final List<Range> ranges = extractExpressionRanges(literalValue);
-        if (ranges.isEmpty()) {
-            return new StringLiteralEvaluator(literalValue);
-        }
-
-        final List<Evaluator<?>> evaluators = new ArrayList<>();
-
-        int lastIndex = 0;
-        for (final Range range : ranges) {
-            if (range.getStart() > lastIndex) {
-                evaluators.add(newStringLiteralEvaluator(literalValue.substring(lastIndex, range.getStart())));
-            }
-
-            final String treeText = literalValue.substring(range.getStart(), range.getEnd() + 1);
-            evaluators.add(buildEvaluator(compileTree(treeText)));
-            lastIndex = range.getEnd() + 1;
-        }
-
-        final Range lastRange = ranges.get(ranges.size() - 1);
-        if (lastRange.getEnd() + 1 < literalValue.length()) {
-            final String treeText = literalValue.substring(lastRange.getEnd() + 1);
-            evaluators.add(newStringLiteralEvaluator(treeText));
-        }
-
-        if (evaluators.size() == 1) {
-            return toStringEvaluator(evaluators.get(0));
-        }
-
-        Evaluator<String> lastEvaluator = toStringEvaluator(evaluators.get(0));
-        for (int i = 1; i < evaluators.size(); i++) {
-            lastEvaluator = new AppendEvaluator(lastEvaluator, toStringEvaluator(evaluators.get(i)));
-        }
-
-        return lastEvaluator;
-    }
-
-    private static Evaluator<?> buildEvaluator(final Tree tree) {
-        switch (tree.getType()) {
-            case EXPRESSION: {
-                return buildExpressionEvaluator(tree);
-            }
-            case ATTRIBUTE_REFERENCE: {
-                final Evaluator<?> childEvaluator = buildEvaluator(tree.getChild(0));
-                if (childEvaluator instanceof MultiAttributeEvaluator) {
-                    return childEvaluator;
-                }
-                return new AttributeEvaluator(toStringEvaluator(childEvaluator));
-            }
-            case MULTI_ATTRIBUTE_REFERENCE: {
-
-                final Tree functionTypeTree = tree.getChild(0);
-                final int multiAttrType = functionTypeTree.getType();
-                if (multiAttrType == ANY_DELINEATED_VALUE || multiAttrType == ALL_DELINEATED_VALUES) {
-                    final Evaluator<String> delineatedValueEvaluator = toStringEvaluator(buildEvaluator(tree.getChild(1)));
-                    final Evaluator<String> delimiterEvaluator = toStringEvaluator(buildEvaluator(tree.getChild(2)));
-
-                    return new DelineatedAttributeEvaluator(delineatedValueEvaluator, delimiterEvaluator, multiAttrType);
-                }
-
-                final List<String> attributeNames = new ArrayList<>();
-                for (int i = 1; i < tree.getChildCount(); i++) {  // skip the first child because that's the name of the multi-attribute function
-                    attributeNames.add(newStringLiteralEvaluator(tree.getChild(i).getText()).evaluate(null).getValue());
-                }
-
-                switch (multiAttrType) {
-                    case ALL_ATTRIBUTES:
-                        for (final String attributeName : attributeNames) {
-                            try {
-                                FlowFile.KeyValidator.validateKey(attributeName);
-                            } catch (final IllegalArgumentException iae) {
-                                throw new AttributeExpressionLanguageParsingException("Invalid Attribute Name: " + attributeName + ". " + iae.getMessage());
-                            }
-                        }
-
-                        return new MultiNamedAttributeEvaluator(attributeNames, ALL_ATTRIBUTES);
-                    case ALL_MATCHING_ATTRIBUTES:
-                        return new MultiMatchAttributeEvaluator(attributeNames, ALL_MATCHING_ATTRIBUTES);
-                    case ANY_ATTRIBUTE:
-                        for (final String attributeName : attributeNames) {
-                            try {
-                                FlowFile.KeyValidator.validateKey(attributeName);
-                            } catch (final IllegalArgumentException iae) {
-                                throw new AttributeExpressionLanguageParsingException("Invalid Attribute Name: " + attributeName + ". " + iae.getMessage());
-                            }
-                        }
-
-                        return new MultiNamedAttributeEvaluator(attributeNames, ANY_ATTRIBUTE);
-                    case ANY_MATCHING_ATTRIBUTE:
-                        return new MultiMatchAttributeEvaluator(attributeNames, ANY_MATCHING_ATTRIBUTE);
-                    default:
-                        throw new AssertionError("Illegal Multi-Attribute Reference: " + functionTypeTree.toString());
-                }
-            }
-            case ATTR_NAME: {
-                return newStringLiteralEvaluator(tree.getChild(0).getText());
-            }
-            case WHOLE_NUMBER: {
-                return new WholeNumberLiteralEvaluator(tree.getText());
-            }
-            case STRING_LITERAL: {
-                return newStringLiteralEvaluator(tree.getText());
-            }
-            case DECIMAL: {
-                return new DecimalLiteralEvaluator(tree.getText());
-            }
-            case TRUE:
-            case FALSE:
-                return buildBooleanEvaluator(tree);
-            case UUID: {
-                return new UuidEvaluator();
-            }
-            case NOW: {
-                return new NowEvaluator();
-            }
-            case TO_LITERAL: {
-                final Evaluator<?> argEvaluator = buildEvaluator(tree.getChild(0));
-                return new ToLiteralEvaluator(argEvaluator);
-            }
-            case IP: {
-                try {
-                    return new IPEvaluator();
-                } catch (final UnknownHostException e) {
-                    throw new AttributeExpressionLanguageException(e);
-                }
-            }
-            case HOSTNAME: {
-                if (tree.getChildCount() == 0) {
-                    try {
-                        return new HostnameEvaluator(false);
-                    } catch (final UnknownHostException e) {
-                        throw new AttributeExpressionLanguageException(e);
-                    }
-                } else if (tree.getChildCount() == 1) {
-                    final Tree childTree = tree.getChild(0);
-                    try {
-                        switch (childTree.getType()) {
-                            case TRUE:
-                                return new HostnameEvaluator(true);
-                            case FALSE:
-                                return new HostnameEvaluator(false);
-                            default:
-                                throw new AttributeExpressionLanguageParsingException("Call to hostname() must take 0 or 1 (boolean) parameter");
-                        }
-                    } catch (final UnknownHostException e) {
-                        throw new AttributeExpressionLanguageException(e);
-                    }
-                } else {
-                    throw new AttributeExpressionLanguageParsingException("Call to hostname() must take 0 or 1 (boolean) parameter");
-                }
-            }
-            case NEXT_INT: {
-                return new OneUpSequenceEvaluator();
-            }
-            case RANDOM: {
-                return new RandomNumberGeneratorEvaluator();
-            }
-            case MATH: {
-                if (tree.getChildCount() == 1) {
-                    return addToken(new MathEvaluator(null, toStringEvaluator(buildEvaluator(tree.getChild(0))), null), "math");
-                } else {
-                    throw new AttributeExpressionLanguageParsingException("Call to math() as the subject must take exactly 1 parameter");
-                }
-            }
-            case GET_STATE_VALUE: {
-                final Tree childTree = tree.getChild(0);
-                final Evaluator<?> argEvaluator = buildEvaluator(childTree);
-                final Evaluator<String> stringEvaluator = toStringEvaluator(argEvaluator);
-                return new GetStateVariableEvaluator(stringEvaluator);
-            }
-            default:
-                throw new AttributeExpressionLanguageParsingException("Unexpected token: " + tree.toString());
-        }
-    }
-
-    private static <T> Evaluator<T> addToken(final Evaluator<T> evaluator, final String token) {
-        evaluator.setToken(token);
-        return evaluator;
-    }
-
-    private static Evaluator<Boolean> buildBooleanEvaluator(final Tree tree) {
-        switch (tree.getType()) {
-            case TRUE:
-                return addToken(new BooleanLiteralEvaluator(true), "true");
-            case FALSE:
-                return addToken(new BooleanLiteralEvaluator(false), "true");
-        }
-        throw new AttributeExpressionLanguageParsingException("Cannot build Boolean evaluator from tree " + tree.toString());
-    }
-
-    @SuppressWarnings({ "rawtypes", "unchecked" })
-    private static Evaluator<?> buildExpressionEvaluator(final Tree tree) {
-        if (tree.getChildCount() == 0) {
-            throw new AttributeExpressionLanguageParsingException("EXPRESSION tree node has no children");
-        }
-
-        final Evaluator<?> evaluator;
-        if (tree.getChildCount() == 1) {
-            evaluator = buildEvaluator(tree.getChild(0));
-        } else {
-            // we can chain together functions in the form of:
-            // ${x:trim():substring(1,2):trim()}
-            // in this case, the subject of the right-most function is the function to its left; its
-            // subject is the function to its left (the first trim()), and its subject is the value of
-            // the 'x' attribute. We accomplish this logic by iterating over all of the children of the
-            // tree from the right-most child going left-ward.
-            evaluator = buildFunctionExpressionEvaluator(tree, 0);
-        }
-
-        Evaluator<?> chosenEvaluator = evaluator;
-        final Evaluator<?> rootEvaluator = getRootSubjectEvaluator(evaluator);
-        if (rootEvaluator != null) {
-            if (rootEvaluator instanceof MultiAttributeEvaluator) {
-                final MultiAttributeEvaluator multiAttrEval = (MultiAttributeEvaluator) rootEvaluator;
-
-                switch (multiAttrEval.getEvaluationType()) {
-                    case ANY_ATTRIBUTE:
-                    case ANY_MATCHING_ATTRIBUTE:
-                    case ANY_DELINEATED_VALUE:
-                        chosenEvaluator = new AnyAttributeEvaluator((BooleanEvaluator) evaluator, multiAttrEval);
-                        break;
-                    case ALL_ATTRIBUTES:
-                    case ALL_MATCHING_ATTRIBUTES:
-                    case ALL_DELINEATED_VALUES: {
-                        final ResultType resultType = evaluator.getResultType();
-                        if (resultType == ResultType.BOOLEAN) {
-                            chosenEvaluator = new AllAttributesEvaluator((BooleanEvaluator) evaluator, multiAttrEval);
-                        } else if (evaluator instanceof ReduceEvaluator) {
-                            chosenEvaluator = new MappingEvaluator((ReduceEvaluator) evaluator, multiAttrEval);
-                        } else {
-                            throw new AttributeExpressionLanguageException("Cannot evaluate Expression because it attempts to reference multiple attributes but does not use a reducing function");
-                        }
-                        break;
-                    }
-                }
-
-                switch (multiAttrEval.getEvaluationType()) {
-                    case ANY_ATTRIBUTE:
-                        chosenEvaluator.setToken("anyAttribute");
-                        break;
-                    case ANY_MATCHING_ATTRIBUTE:
-                        chosenEvaluator.setToken("anyMatchingAttribute");
-                        break;
-                    case ANY_DELINEATED_VALUE:
-                        chosenEvaluator.setToken("anyDelineatedValue");
-                        break;
-                    case ALL_ATTRIBUTES:
-                        chosenEvaluator.setToken("allAttributes");
-                        break;
-                    case ALL_MATCHING_ATTRIBUTES:
-                        chosenEvaluator.setToken("allMatchingAttributes");
-                        break;
-                    case ALL_DELINEATED_VALUES:
-                        chosenEvaluator.setToken("allDelineatedValues");
-                        break;
-                }
-            }
-        }
-
-        return chosenEvaluator;
-    }
-
-    private static Evaluator<?> buildFunctionExpressionEvaluator(final Tree tree, final int offset) {
-        if (tree.getChildCount() == 0) {
-            throw new AttributeExpressionLanguageParsingException("EXPRESSION tree node has no children");
-        }
-        final int firstChildIndex = tree.getChildCount() - offset - 1;
-        if (firstChildIndex == 0) {
-            return buildEvaluator(tree.getChild(0));
-        }
 
-        final Tree functionTree = tree.getChild(firstChildIndex);
-        final Evaluator<?> subjectEvaluator = buildFunctionExpressionEvaluator(tree, offset + 1);
-
-        final Tree functionNameTree = functionTree.getChild(0);
-        final List<Evaluator<?>> argEvaluators = new ArrayList<>();
-        for (int i = 1; i < functionTree.getChildCount(); i++) {
-            argEvaluators.add(buildEvaluator(functionTree.getChild(i)));
-        }
-        return buildFunctionEvaluator(functionNameTree, subjectEvaluator, argEvaluators);
-    }
-
-    private static List<Evaluator<?>> verifyArgCount(final List<Evaluator<?>> args, final int count, final String functionName) {
-        if (args.size() != count) {
-            throw new AttributeExpressionLanguageParsingException(functionName + "() function takes " + count + " arguments");
-        }
-        return args;
-    }
-
-    private static Evaluator<String> toStringEvaluator(final Evaluator<?> evaluator) {
-        return toStringEvaluator(evaluator, null);
-    }
-
-    private static Evaluator<String> toStringEvaluator(final Evaluator<?> evaluator, final String location) {
-        if (evaluator.getResultType() == ResultType.STRING) {
-            return (StringEvaluator) evaluator;
-        }
-
-        return addToken(new StringCastEvaluator(evaluator), evaluator.getToken());
-    }
-
-    @SuppressWarnings("unchecked")
-    private static Evaluator<Boolean> toBooleanEvaluator(final Evaluator<?> evaluator, final String location) {
-        switch (evaluator.getResultType()) {
-            case BOOLEAN:
-                return (Evaluator<Boolean>) evaluator;
-            case STRING:
-                return addToken(new BooleanCastEvaluator((StringEvaluator) evaluator), evaluator.getToken());
-            default:
-                throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + evaluator.getResultType() + " to " + ResultType.BOOLEAN
-                    + (location == null ? "" : " at location [" + location + "]"));
-        }
-
-    }
-
-    private static Evaluator<Boolean> toBooleanEvaluator(final Evaluator<?> evaluator) {
-        return toBooleanEvaluator(evaluator, null);
-    }
-
-    private static Evaluator<Long> toWholeNumberEvaluator(final Evaluator<?> evaluator) {
-        return toWholeNumberEvaluator(evaluator, null);
-    }
-
-    @SuppressWarnings("unchecked")
-    private static Evaluator<Long> toWholeNumberEvaluator(final Evaluator<?> evaluator, final String location) {
-        switch (evaluator.getResultType()) {
-            case WHOLE_NUMBER:
-                return (Evaluator<Long>) evaluator;
-            case STRING:
-            case DATE:
-            case DECIMAL:
-            case NUMBER:
-                return addToken(new WholeNumberCastEvaluator(evaluator), evaluator.getToken());
-            default:
-                throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + evaluator.getResultType() + " to " + ResultType.WHOLE_NUMBER
-                    + (location == null ? "" : " at location [" + location + "]"));
-        }
-    }
-
-    private static Evaluator<Double> toDecimalEvaluator(final Evaluator<?> evaluator) {
-        return toDecimalEvaluator(evaluator, null);
-    }
-
-    @SuppressWarnings("unchecked")
-    private static Evaluator<Double> toDecimalEvaluator(final Evaluator<?> evaluator, final String location) {
-        switch (evaluator.getResultType()) {
-            case DECIMAL:
-                return (Evaluator<Double>) evaluator;
-            case WHOLE_NUMBER:
-            case STRING:
-            case DATE:
-            case NUMBER:
-                return addToken(new DecimalCastEvaluator(evaluator), evaluator.getToken());
-            default:
-                throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + evaluator.getResultType() + " to " + ResultType.DECIMAL
-                        + (location == null ? "" : " at location [" + location + "]"));
-        }
-    }
-
-    private static Evaluator<Number> toNumberEvaluator(final Evaluator<?> evaluator) {
-        return toNumberEvaluator(evaluator, null);
-    }
-
-    @SuppressWarnings("unchecked")
-    private static Evaluator<Number> toNumberEvaluator(final Evaluator<?> evaluator, final String location) {
-        switch (evaluator.getResultType()) {
-            case NUMBER:
-                return (Evaluator<Number>) evaluator;
-            case STRING:
-            case DATE:
-            case DECIMAL:
-            case WHOLE_NUMBER:
-                return addToken(new NumberCastEvaluator(evaluator), evaluator.getToken());
-            default:
-                throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + evaluator.getResultType() + " to " + ResultType.WHOLE_NUMBER
-                        + (location == null ? "" : " at location [" + location + "]"));
-        }
-    }
-
-    private static DateEvaluator toDateEvaluator(final Evaluator<?> evaluator) {
-        return toDateEvaluator(evaluator, null);
-    }
-
-    private static DateEvaluator toDateEvaluator(final Evaluator<?> evaluator, final String location) {
-        if (evaluator.getResultType() == ResultType.DATE) {
-            return (DateEvaluator) evaluator;
-        }
-
-        return new DateCastEvaluator(evaluator);
-    }
-
-    private static Evaluator<?> buildFunctionEvaluator(final Tree tree, final Evaluator<?> subjectEvaluator, final List<Evaluator<?>> argEvaluators) {
-        switch (tree.getType()) {
-            case TRIM: {
-                verifyArgCount(argEvaluators, 0, "trim");
-                return addToken(new TrimEvaluator(toStringEvaluator(subjectEvaluator)), "trim");
-            }
-            case TO_STRING: {
-                verifyArgCount(argEvaluators, 0, "toString");
-                return addToken(new ToStringEvaluator(subjectEvaluator), "toString");
-            }
-            case TO_LOWER: {
-                verifyArgCount(argEvaluators, 0, "toLower");
-                return addToken(new ToLowerEvaluator(toStringEvaluator(subjectEvaluator)), "toLower");
-            }
-            case TO_UPPER: {
-                verifyArgCount(argEvaluators, 0, "toUpper");
-                return addToken(new ToUpperEvaluator(toStringEvaluator(subjectEvaluator)), "toUpper");
-            }
-            case URL_ENCODE: {
-                verifyArgCount(argEvaluators, 0, "urlEncode");
-                return addToken(new UrlEncodeEvaluator(toStringEvaluator(subjectEvaluator)), "urlEncode");
-            }
-            case URL_DECODE: {
-                verifyArgCount(argEvaluators, 0, "urlDecode");
-                return addToken(new UrlDecodeEvaluator(toStringEvaluator(subjectEvaluator)), "urlDecode");
-            }
-            case BASE64_ENCODE: {
-                verifyArgCount(argEvaluators, 0, "base64Encode");
-                return addToken(new Base64EncodeEvaluator(toStringEvaluator(subjectEvaluator)), "base64Encode");
-            }
-            case BASE64_DECODE: {
-                verifyArgCount(argEvaluators, 0, "base64Decode");
-                return addToken(new Base64DecodeEvaluator(toStringEvaluator(subjectEvaluator)), "base64Decode");
-            }
-            case ESCAPE_CSV: {
-                verifyArgCount(argEvaluators, 0, "escapeCsv");
-                return addToken(CharSequenceTranslatorEvaluator.csvEscapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
-            }
-            case ESCAPE_HTML3: {
-                verifyArgCount(argEvaluators, 0, "escapeHtml3");
-                return addToken(CharSequenceTranslatorEvaluator.html3EscapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
-            }
-            case ESCAPE_HTML4: {
-                verifyArgCount(argEvaluators, 0, "escapeHtml4");
-                return addToken(CharSequenceTranslatorEvaluator.html4EscapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
-            }
-            case ESCAPE_JSON: {
-                verifyArgCount(argEvaluators, 0, "escapeJson");
-                return addToken(CharSequenceTranslatorEvaluator.jsonEscapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
-            }
-            case ESCAPE_XML: {
-                verifyArgCount(argEvaluators, 0, "escapeXml");
-                return addToken(CharSequenceTranslatorEvaluator.xmlEscapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
-            }
-            case UNESCAPE_CSV: {
-                verifyArgCount(argEvaluators, 0, "unescapeCsv");
-                return addToken(CharSequenceTranslatorEvaluator.csvUnescapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
-            }
-            case UNESCAPE_HTML3: {
-                verifyArgCount(argEvaluators, 0, "unescapeHtml3");
-                return addToken(CharSequenceTranslatorEvaluator.html3UnescapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
-            }
-            case UNESCAPE_HTML4: {
-                verifyArgCount(argEvaluators, 0, "unescapeHtml4");
-                return addToken(CharSequenceTranslatorEvaluator.html4UnescapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
-            }
-            case UNESCAPE_JSON: {
-                verifyArgCount(argEvaluators, 0, "unescapeJson");
-                return addToken(CharSequenceTranslatorEvaluator.jsonUnescapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
-            }
-            case UNESCAPE_XML: {
-                verifyArgCount(argEvaluators, 0, "unescapeXml");
-                return addToken(CharSequenceTranslatorEvaluator.xmlUnescapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
-            }
-            case SUBSTRING_BEFORE: {
-                verifyArgCount(argEvaluators, 1, "substringBefore");
-                return addToken(new SubstringBeforeEvaluator(toStringEvaluator(subjectEvaluator),
-                    toStringEvaluator(argEvaluators.get(0), "first argument to substringBefore")), "substringBefore");
-            }
-            case SUBSTRING_BEFORE_LAST: {
-                verifyArgCount(argEvaluators, 1, "substringBeforeLast");
-                return addToken(new SubstringBeforeLastEvaluator(toStringEvaluator(subjectEvaluator),
-                    toStringEvaluator(argEvaluators.get(0), "first argument to substringBeforeLast")), "substringBeforeLast");
-            }
-            case SUBSTRING_AFTER: {
-                verifyArgCount(argEvaluators, 1, "substringAfter");
-                return addToken(new SubstringAfterEvaluator(toStringEvaluator(subjectEvaluator),
-                    toStringEvaluator(argEvaluators.get(0), "first argument to substringAfter")), "substringAfter");
-            }
-            case SUBSTRING_AFTER_LAST: {
-                verifyArgCount(argEvaluators, 1, "substringAfterLast");
-                return addToken(new SubstringAfterLastEvaluator(toStringEvaluator(subjectEvaluator),
-                    toStringEvaluator(argEvaluators.get(0), "first argument to substringAfterLast")), "substringAfterLast");
-            }
-            case REPLACE_NULL: {
-                verifyArgCount(argEvaluators, 1, "replaceNull");
-                return addToken(new ReplaceNullEvaluator(toStringEvaluator(subjectEvaluator),
-                    toStringEvaluator(argEvaluators.get(0), "first argument to replaceNull")), "replaceNull");
-            }
-            case REPLACE_EMPTY: {
-                verifyArgCount(argEvaluators, 1, "replaceEmtpy");
-                return addToken(new ReplaceEmptyEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0), "first argument to replaceEmpty")), "replaceEmpty");
-            }
-            case REPLACE: {
-                verifyArgCount(argEvaluators, 2, "replace");
-                return addToken(new ReplaceEvaluator(toStringEvaluator(subjectEvaluator),
-                    toStringEvaluator(argEvaluators.get(0), "first argument to replace"),
-                    toStringEvaluator(argEvaluators.get(1), "second argument to replace")), "replace");
-            }
-            case REPLACE_FIRST: {
-                verifyArgCount(argEvaluators, 2, "replaceFirst");
-                return addToken(new ReplaceFirstEvaluator(toStringEvaluator(subjectEvaluator),
-                        toStringEvaluator(argEvaluators.get(0), "first argument to replaceFirst"),
-                        toStringEvaluator(argEvaluators.get(1), "second argument to replaceFirst")), "replaceFirst");
-            }
-            case REPLACE_ALL: {
-                verifyArgCount(argEvaluators, 2, "replaceAll");
-                return addToken(new ReplaceAllEvaluator(toStringEvaluator(subjectEvaluator),
-                    toStringEvaluator(argEvaluators.get(0), "first argument to replaceAll"),
-                    toStringEvaluator(argEvaluators.get(1), "second argument to replaceAll")), "replaceAll");
-            }
-            case APPEND: {
-                verifyArgCount(argEvaluators, 1, "append");
-                return addToken(new AppendEvaluator(toStringEvaluator(subjectEvaluator),
-                    toStringEvaluator(argEvaluators.get(0), "first argument to append")), "append");
-            }
-            case PREPEND: {
-                verifyArgCount(argEvaluators, 1, "prepend");
-                return addToken(new PrependEvaluator(toStringEvaluator(subjectEvaluator),
-                    toStringEvaluator(argEvaluators.get(0), "first argument to prepend")), "prepend");
-            }
-            case SUBSTRING: {
-                final int numArgs = argEvaluators.size();
-                if (numArgs == 1) {
-                    return addToken(new SubstringEvaluator(toStringEvaluator(subjectEvaluator),
-                        toWholeNumberEvaluator(argEvaluators.get(0), "first argument to substring")), "substring");
-                } else if (numArgs == 2) {
-                    return addToken(new SubstringEvaluator(toStringEvaluator(subjectEvaluator),
-                        toWholeNumberEvaluator(argEvaluators.get(0), "first argument to substring"),
-                        toWholeNumberEvaluator(argEvaluators.get(1), "second argument to substring")), "substring");
-                } else {
-                    throw new AttributeExpressionLanguageParsingException("substring() function can take either 1 or 2 arguments but cannot take " + numArgs + " arguments");
-                }
-            }
-            case JOIN: {
-                verifyArgCount(argEvaluators, 1, "join");
-                return addToken(new JoinEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0))), "join");
-            }
-            case COUNT: {
-                verifyArgCount(argEvaluators, 0, "count");
-                return addToken(new CountEvaluator(subjectEvaluator), "count");
-            }
-            case IS_NULL: {
-                verifyArgCount(argEvaluators, 0, "isNull");
-                return addToken(new IsNullEvaluator(toStringEvaluator(subjectEvaluator)), "isNull");
-            }
-            case IS_EMPTY: {
-                verifyArgCount(argEvaluators, 0, "isEmpty");
-                return addToken(new IsEmptyEvaluator(toStringEvaluator(subjectEvaluator)), "isEmpty");
-            }
-            case NOT_NULL: {
-                verifyArgCount(argEvaluators, 0, "notNull");
-                return addToken(new NotNullEvaluator(toStringEvaluator(subjectEvaluator)), "notNull");
-            }
-            case STARTS_WITH: {
-                verifyArgCount(argEvaluators, 1, "startsWith");
-                return addToken(new StartsWithEvaluator(toStringEvaluator(subjectEvaluator),
-                    toStringEvaluator(argEvaluators.get(0), "first argument to startsWith")), "startsWith");
-            }
-            case ENDS_WITH: {
-                verifyArgCount(argEvaluators, 1, "endsWith");
-                return addToken(new EndsWithEvaluator(toStringEvaluator(subjectEvaluator),
-                    toStringEvaluator(argEvaluators.get(0), "first argument to endsWith")), "endsWith");
-            }
-            case CONTAINS: {
-                verifyArgCount(argEvaluators, 1, "contains");
-                return addToken(new ContainsEvaluator(toStringEvaluator(subjectEvaluator),
-                    toStringEvaluator(argEvaluators.get(0), "first argument to contains")), "contains");
-            }
-            case IN: {
-                List<Evaluator<String>> list = new ArrayList<Evaluator<String>>();
-                for(int i = 0; i < argEvaluators.size(); i++) {
-                    list.add(toStringEvaluator(argEvaluators.get(i), i + "th argument to in"));
-                }
-                return addToken(new InEvaluator(toStringEvaluator(subjectEvaluator), list), "in");
-            }
-            case FIND: {
-                verifyArgCount(argEvaluators, 1, "find");
-                return addToken(new FindEvaluator(toStringEvaluator(subjectEvaluator),
-                    toStringEvaluator(argEvaluators.get(0), "first argument to find")), "find");
-            }
-            case MATCHES: {
-                verifyArgCount(argEvaluators, 1, "matches");
-                return addToken(new MatchesEvaluator(toStringEvaluator(subjectEvaluator),
-                    toStringEvaluator(argEvaluators.get(0), "first argument to matches")), "matches");
-            }
-            case EQUALS: {
-                verifyArgCount(argEvaluators, 1, "equals");
-                return addToken(new EqualsEvaluator(subjectEvaluator, argEvaluators.get(0)), "equals");
-            }
-            case EQUALS_IGNORE_CASE: {
-                verifyArgCount(argEvaluators, 1, "equalsIgnoreCase");
-                return addToken(new EqualsIgnoreCaseEvaluator(toStringEvaluator(subjectEvaluator),
-                    toStringEvaluator(argEvaluators.get(0), "first argument to equalsIgnoreCase")), "equalsIgnoreCase");
-            }
-            case GREATER_THAN: {
-                verifyArgCount(argEvaluators, 1, "gt");
-                return addToken(new GreaterThanEvaluator(toNumberEvaluator(subjectEvaluator),
-                    toNumberEvaluator(argEvaluators.get(0), "first argument to gt")), "gt");
-            }
-            case GREATER_THAN_OR_EQUAL: {
-                verifyArgCount(argEvaluators, 1, "ge");
-                return addToken(new GreaterThanOrEqualEvaluator(toNumberEvaluator(subjectEvaluator),
-                    toNumberEvaluator(argEvaluators.get(0), "first argument to ge")), "ge");
-            }
-            case LESS_THAN: {
-                verifyArgCount(argEvaluators, 1, "lt");
-                return addToken(new LessThanEvaluator(toNumberEvaluator(subjectEvaluator),
-                    toNumberEvaluator(argEvaluators.get(0), "first argument to lt")), "lt");
-            }
-            case LESS_THAN_OR_EQUAL: {
-                verifyArgCount(argEvaluators, 1, "le");
-                return addToken(new LessThanOrEqualEvaluator(toNumberEvaluator(subjectEvaluator),
-                    toNumberEvaluator(argEvaluators.get(0), "first argument to le")), "le");
-            }
-            case LENGTH: {
-                verifyArgCount(argEvaluators, 0, "length");
-                return addToken(new LengthEvaluator(toStringEvaluator(subjectEvaluator)), "length");
-            }
-            case TO_DATE: {
-                if (argEvaluators.isEmpty()) {
-                    return addToken(new NumberToDateEvaluator(toWholeNumberEvaluator(subjectEvaluator)), "toDate");
-                } else if (subjectEvaluator.getResultType() == ResultType.STRING && argEvaluators.size() == 1) {
-                    return addToken(new StringToDateEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0)), null), "toDate");
-                } else if (subjectEvaluator.getResultType() == ResultType.STRING && argEvaluators.size() == 2) {
-                    return addToken(new StringToDateEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0)), toStringEvaluator(argEvaluators.get(1))), "toDate");
-                } else {
-                    return addToken(new NumberToDateEvaluator(toWholeNumberEvaluator(subjectEvaluator)), "toDate");
-                }
-            }
-            case TO_NUMBER: {
-                verifyArgCount(argEvaluators, 0, "toNumber");
-                switch (subjectEvaluator.getResultType()) {
-                    case STRING:
-                    case WHOLE_NUMBER:
-                    case DECIMAL:
-                    case NUMBER:
-                    case DATE:
-                        return addToken(toWholeNumberEvaluator(subjectEvaluator), "toNumber");
-                    default:
-                        throw new AttributeExpressionLanguageParsingException(subjectEvaluator + " returns type " + subjectEvaluator.getResultType() + " but expected to get " + ResultType.STRING +
-                                ", " + ResultType.DECIMAL + ", or " + ResultType.DATE);
-                }
-            }
-            case TO_DECIMAL: {
-                verifyArgCount(argEvaluators, 0, "toDecimal");
-                switch (subjectEvaluator.getResultType()) {
-                    case WHOLE_NUMBER:
-                    case DECIMAL:
-                    case STRING:
-                    case NUMBER:
-                    case DATE:
-                        return addToken(toDecimalEvaluator(subjectEvaluator), "toDecimal");
-                    default:
-                        throw new AttributeExpressionLanguageParsingException(subjectEvaluator + " returns type " + subjectEvaluator.getResultType() + " but expected to get " + ResultType.STRING +
-                                ", " + ResultType.WHOLE_NUMBER + ", or " + ResultType.DATE);
-                }
-            }
-            case TO_RADIX: {
-                if (argEvaluators.size() == 1) {
-                    return addToken(new ToRadixEvaluator(toWholeNumberEvaluator(subjectEvaluator),
-                            toWholeNumberEvaluator(argEvaluators.get(0))), "toRadix");
-                } else {
-                    return addToken(new ToRadixEvaluator(toWholeNumberEvaluator(subjectEvaluator),
-                            toWholeNumberEvaluator(argEvaluators.get(0)), toWholeNumberEvaluator(argEvaluators.get(1))), "toRadix");
-                }
-            }
-            case FROM_RADIX: {
-                return addToken(new FromRadixEvaluator(toStringEvaluator(subjectEvaluator),
-                        toWholeNumberEvaluator(argEvaluators.get(0))), "fromRadix");
-            }
-            case MOD: {
-                return addToken(new ModEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "mod");
-            }
-            case PLUS: {
-                return addToken(new PlusEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "plus");
-            }
-            case MINUS: {
-                return addToken(new MinusEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "minus");
-            }
-            case MULTIPLY: {
-                return addToken(new MultiplyEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "multiply");
-            }
-            case DIVIDE: {
-                return addToken(new DivideEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "divide");
-            }
-            case MATH: {
-                if (argEvaluators.size() == 1) {
-                    return addToken(new MathEvaluator(toNumberEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0)), null), "math");
-                } else if (argEvaluators.size() == 2){
-                    return addToken(new MathEvaluator(toNumberEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0)), toNumberEvaluator(argEvaluators.get(1))), "math");
-                } else {
-                    throw new AttributeExpressionLanguageParsingException("math() function takes 1 or 2 arguments");
-                }
-            }
-            case RANDOM : {
-                return addToken(new RandomNumberGeneratorEvaluator(), "random");
-            }
-            case INDEX_OF: {
-                verifyArgCount(argEvaluators, 1, "indexOf");
-                return addToken(new IndexOfEvaluator(toStringEvaluator(subjectEvaluator),
-                    toStringEvaluator(argEvaluators.get(0), "first argument to indexOf")), "indexOf");
-            }
-            case LAST_INDEX_OF: {
-                verifyArgCount(argEvaluators, 1, "lastIndexOf");
-                return addToken(new LastIndexOfEvaluator(toStringEvaluator(subjectEvaluator),
-                    toStringEvaluator(argEvaluators.get(0), "first argument to lastIndexOf")), "lastIndexOf");
-            }
-            case FORMAT: {
-                if(argEvaluators.size() == 1) {
-                    return addToken(new FormatEvaluator(toDateEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0), "first argument of format"), null), "format");
-                } else if (argEvaluators.size() == 2) {
-                    return addToken(new FormatEvaluator(toDateEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0)), toStringEvaluator(argEvaluators.get(1))), "format");
-                } else {
-                    throw new AttributeExpressionLanguageParsingException("format() function takes 1 or 2 arguments");
-                }
-            }
-            case OR: {
-                return addToken(new OrEvaluator(toBooleanEvaluator(subjectEvaluator), toBooleanEvaluator(argEvaluators.get(0))), "or");
-            }
-            case AND: {
-                return addToken(new AndEvaluator(toBooleanEvaluator(subjectEvaluator), toBooleanEvaluator(argEvaluators.get(0))), "and");
-            }
-            case NOT: {
-                return addToken(new NotEvaluator(toBooleanEvaluator(subjectEvaluator)), "not");
-            }
-            case GET_DELIMITED_FIELD: {
-                if (argEvaluators.size() == 1) {
-                    // Only a single argument - the index to return.
-                    return addToken(new GetDelimitedFieldEvaluator(toStringEvaluator(subjectEvaluator),
-                        toWholeNumberEvaluator(argEvaluators.get(0), "first argument of getDelimitedField")), "getDelimitedField");
-                } else if (argEvaluators.size() == 2) {
-                    // two arguments - index and delimiter.
-                    return addToken(new GetDelimitedFieldEvaluator(toStringEvaluator(subjectEvaluator),
-                        toWholeNumberEvaluator(argEvaluators.get(0), "first argument of getDelimitedField"),
-                        toStringEvaluator(argEvaluators.get(1), "second argument of getDelimitedField")),
-                        "getDelimitedField");
-                } else if (argEvaluators.size() == 3) {
-                    // 3 arguments - index, delimiter, quote char.
-                    return addToken(new GetDelimitedFieldEvaluator(toStringEvaluator(subjectEvaluator),
-                        toWholeNumberEvaluator(argEvaluators.get(0), "first argument of getDelimitedField"),
-                        toStringEvaluator(argEvaluators.get(1), "second argument of getDelimitedField"),
-                        toStringEvaluator(argEvaluators.get(2), "third argument of getDelimitedField")),
-                        "getDelimitedField");
-                } else if (argEvaluators.size() == 4) {
-                    // 4 arguments - index, delimiter, quote char, escape char
-                    return addToken(new GetDelimitedFieldEvaluator(toStringEvaluator(subjectEvaluator),
-                        toWholeNumberEvaluator(argEvaluators.get(0), "first argument of getDelimitedField"),
-                        toStringEvaluator(argEvaluators.get(1), "second argument of getDelimitedField"),
-                        toStringEvaluator(argEvaluators.get(2), "third argument of getDelimitedField"),
-                        toStringEvaluator(argEvaluators.get(3), "fourth argument of getDelimitedField")),
-                        "getDelimitedField");
-                } else {
-                    // 5 arguments - index, delimiter, quote char, escape char, strip escape/quote chars flag
-                    return addToken(new GetDelimitedFieldEvaluator(toStringEvaluator(subjectEvaluator),
-                        toWholeNumberEvaluator(argEvaluators.get(0), "first argument of getDelimitedField"),
-                        toStringEvaluator(argEvaluators.get(1), "second argument of getDelimitedField"),
-                        toStringEvaluator(argEvaluators.get(2), "third argument of getDelimitedField"),
-                        toStringEvaluator(argEvaluators.get(3), "fourth argument of getDelimitedField"),
-                        toBooleanEvaluator(argEvaluators.get(4), "fifth argument of getDelimitedField")),
-                        "getDelimitedField");
-                }
-            }
-            case JSON_PATH: {
-                verifyArgCount(argEvaluators, 1, "jsonPath");
-                return addToken(new JsonPathEvaluator(toStringEvaluator(subjectEvaluator),
-                        toStringEvaluator(argEvaluators.get(0), "first argument to jsonPath")), "jsonPath");
-            }
-            case IF_ELSE: {
-                verifyArgCount(argEvaluators, 2, "ifElse");
-                return addToken(new IfElseEvaluator(toBooleanEvaluator(subjectEvaluator),
-                        toStringEvaluator(argEvaluators.get(0), "argument to return if true"),
-                        toStringEvaluator(argEvaluators.get(1), "argument to return if false")), "ifElse");
-            }
-            default:
-                throw new AttributeExpressionLanguageParsingException("Expected a Function-type expression but got " + tree.toString());
-            }
-    }
 
     public static class Range {
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java
index 9f12c92..cdf5a2d 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java
@@ -16,36 +16,45 @@
  */
 package org.apache.nifi.attribute.expression.language;
 
-import java.util.ArrayList;
-import java.util.HashMap;
+import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
+import org.apache.nifi.attribute.expression.language.compile.CompiledExpression;
+import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.literals.StringLiteralEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.AllAttributesEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.AnyAttributeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.AttributeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.MappingEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiAttributeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiMatchAttributeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiNamedAttributeEvaluator;
 import org.apache.nifi.expression.AttributeValueDecorator;
 import org.apache.nifi.processor.exception.ProcessException;
 
-import org.antlr.runtime.tree.Tree;
-
 public class StandardPreparedQuery implements PreparedQuery {
 
     private final List<String> queryStrings;
-    private final Map<String, Tree> trees;
+    private final Map<String, CompiledExpression> expressions;
+    private volatile VariableImpact variableImpact;
 
-    public StandardPreparedQuery(final List<String> queryStrings, final Map<String, Tree> trees) {
-        this.queryStrings = new ArrayList<>(queryStrings);
-        this.trees = new HashMap<>(trees);
+    public StandardPreparedQuery(final List<String> queryStrings, final Map<String, CompiledExpression> expressions) {
+        this.queryStrings = queryStrings;
+        this.expressions = expressions;
     }
 
-
     @Override
     public String evaluateExpressions(final Map<String, String> valMap, final AttributeValueDecorator decorator, final Map<String, String> stateVariables) throws ProcessException {
         final StringBuilder sb = new StringBuilder();
         for (final String val : queryStrings) {
-            final Tree tree = trees.get(val);
-            if (tree == null) {
+            final CompiledExpression expression = expressions.get(val);
+            if (expression == null) {
                 sb.append(val);
             } else {
-                final String evaluated = Query.evaluateExpression(tree, val, valMap, decorator, stateVariables);
+                final String evaluated = Query.evaluateExpression(expression.getTree(), val, valMap, decorator, stateVariables);
                 if (evaluated != null) {
                     sb.append(evaluated);
                 }
@@ -62,6 +71,56 @@ public class StandardPreparedQuery implements PreparedQuery {
 
     @Override
     public boolean isExpressionLanguagePresent() {
-        return !trees.isEmpty();
+        return !expressions.isEmpty();
+    }
+
+    @Override
+    public VariableImpact getVariableImpact() {
+        final VariableImpact existing = this.variableImpact;
+        if (existing != null) {
+            return existing;
+        }
+
+        final Set<String> variables = new HashSet<>();
+
+        for (final CompiledExpression expression : expressions.values()) {
+            for (final Evaluator<?> evaluator : expression.getAllEvaluators()) {
+                if (evaluator instanceof AttributeEvaluator) {
+                    final AttributeEvaluator attributeEval = (AttributeEvaluator) evaluator;
+                    final Evaluator<String> nameEval = attributeEval.getNameEvaluator();
+
+                    if (nameEval instanceof StringLiteralEvaluator) {
+                        final String referencedVar = nameEval.evaluate(Collections.emptyMap()).getValue();
+                        variables.add(referencedVar);
+                    }
+                } else if (evaluator instanceof AllAttributesEvaluator) {
+                    final AllAttributesEvaluator allAttrsEval = (AllAttributesEvaluator) evaluator;
+                    final MultiAttributeEvaluator iteratingEval = allAttrsEval.getVariableIteratingEvaluator();
+                    if (iteratingEval instanceof MultiNamedAttributeEvaluator) {
+                        variables.addAll(((MultiNamedAttributeEvaluator) iteratingEval).getAttributeNames());
+                    } else if (iteratingEval instanceof MultiMatchAttributeEvaluator) {
+                        return VariableImpact.ALWAYS_IMPACTED;
+                    }
+                } else if (evaluator instanceof AnyAttributeEvaluator) {
+                    final AnyAttributeEvaluator allAttrsEval = (AnyAttributeEvaluator) evaluator;
+                    final MultiAttributeEvaluator iteratingEval = allAttrsEval.getVariableIteratingEvaluator();
+                    if (iteratingEval instanceof MultiNamedAttributeEvaluator) {
+                        variables.addAll(((MultiNamedAttributeEvaluator) iteratingEval).getAttributeNames());
+                    } else if (iteratingEval instanceof MultiMatchAttributeEvaluator) {
+                        return VariableImpact.ALWAYS_IMPACTED;
+                    }
+                } else if (evaluator instanceof MappingEvaluator) {
+                    final MappingEvaluator<?> allAttrsEval = (MappingEvaluator<?>) evaluator;
+                    final MultiAttributeEvaluator iteratingEval = allAttrsEval.getVariableIteratingEvaluator();
+                    if (iteratingEval instanceof MultiNamedAttributeEvaluator) {
+                        variables.addAll(((MultiNamedAttributeEvaluator) iteratingEval).getAttributeNames());
+                    }
+                }
+            }
+        }
+
+        final VariableImpact impact = new NamedVariableImpact(variables);
+        this.variableImpact = impact;
+        return impact;
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/ValueLookup.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/ValueLookup.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/ValueLookup.java
index 5b0cdda..06c1877 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/ValueLookup.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/ValueLookup.java
@@ -49,6 +49,7 @@ final class ValueLookup implements Map<String, String> {
      * @param flowFile the flowFile to pull attributes from; may be null
      * @param additionalMaps the maps to pull values from; may be null or empty
      */
+    @SuppressWarnings("unchecked")
     ValueLookup(final VariableRegistry registry, final FlowFile flowFile, final Map<String, String>... additionalMaps) {
         for (final Map<String, String> map : additionalMaps) {
             if (map != null && !map.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/VariableImpact.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/VariableImpact.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/VariableImpact.java
new file mode 100644
index 0000000..4a66c87
--- /dev/null
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/VariableImpact.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.attribute.expression.language;
+
+public interface VariableImpact {
+    boolean isImpacted(String variableName);
+
+    public static final VariableImpact NEVER_IMPACTED = var -> false;
+
+    public static final VariableImpact ALWAYS_IMPACTED = var -> true;
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/CompiledExpression.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/CompiledExpression.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/CompiledExpression.java
new file mode 100644
index 0000000..d234445
--- /dev/null
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/CompiledExpression.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.attribute.expression.language.compile;
+
+import java.util.Set;
+
+import org.antlr.runtime.tree.Tree;
+import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
+
+public class CompiledExpression {
+    private final Evaluator<?> rootEvaluator;
+    private final Tree tree;
+    private final String expression;
+    private final Set<Evaluator<?>> allEvaluators;
+
+    public CompiledExpression(final String expression, final Evaluator<?> rootEvaluator, final Tree tree, final Set<Evaluator<?>> allEvaluators) {
+        this.rootEvaluator = rootEvaluator;
+        this.tree = tree;
+        this.expression = expression;
+        this.allEvaluators = allEvaluators;
+    }
+
+    public Evaluator<?> getRootEvaluator() {
+        return rootEvaluator;
+    }
+
+    public Tree getTree() {
+        return tree;
+    }
+
+    public String getExpression() {
+        return expression;
+    }
+
+    public Set<Evaluator<?>> getAllEvaluators() {
+        return allEvaluators;
+    }
+}


[3/9] nifi git commit: NIFI-4224: - Initial implementation of Process Group level Variable Registry - Updated to incorporate PR Feedback - Changed log message because slf4j-simple apparently has a memory leak; passing a String instead of passing in the C

Posted by mc...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
index 36a9524..35686a5 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
@@ -16,7 +16,31 @@
  */
 package org.apache.nifi.web;
 
-import com.google.common.collect.Sets;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.Response;
+
 import org.apache.nifi.action.Action;
 import org.apache.nifi.action.Component;
 import org.apache.nifi.action.FlowChangeAction;
@@ -84,6 +108,7 @@ import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.history.History;
 import org.apache.nifi.history.HistoryQuery;
 import org.apache.nifi.history.PreviousValue;
+import org.apache.nifi.registry.ComponentVariableRegistry;
 import org.apache.nifi.remote.RootGroupPort;
 import org.apache.nifi.reporting.Bulletin;
 import org.apache.nifi.reporting.BulletinQuery;
@@ -92,6 +117,7 @@ import org.apache.nifi.reporting.ComponentType;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.api.dto.AccessPolicyDTO;
 import org.apache.nifi.web.api.dto.AccessPolicySummaryDTO;
+import org.apache.nifi.web.api.dto.AffectedComponentDTO;
 import org.apache.nifi.web.api.dto.BulletinBoardDTO;
 import org.apache.nifi.web.api.dto.BulletinDTO;
 import org.apache.nifi.web.api.dto.BulletinQueryDTO;
@@ -137,6 +163,7 @@ import org.apache.nifi.web.api.dto.SystemDiagnosticsDTO;
 import org.apache.nifi.web.api.dto.TemplateDTO;
 import org.apache.nifi.web.api.dto.UserDTO;
 import org.apache.nifi.web.api.dto.UserGroupDTO;
+import org.apache.nifi.web.api.dto.VariableRegistryDTO;
 import org.apache.nifi.web.api.dto.action.HistoryDTO;
 import org.apache.nifi.web.api.dto.action.HistoryQueryDTO;
 import org.apache.nifi.web.api.dto.flow.FlowDTO;
@@ -157,6 +184,7 @@ import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
 import org.apache.nifi.web.api.entity.AccessPolicyEntity;
 import org.apache.nifi.web.api.entity.AccessPolicySummaryEntity;
 import org.apache.nifi.web.api.entity.ActionEntity;
+import org.apache.nifi.web.api.entity.ActivateControllerServicesEntity;
 import org.apache.nifi.web.api.entity.BulletinEntity;
 import org.apache.nifi.web.api.entity.ComponentReferenceEntity;
 import org.apache.nifi.web.api.entity.ConnectionEntity;
@@ -190,6 +218,8 @@ import org.apache.nifi.web.api.entity.TemplateEntity;
 import org.apache.nifi.web.api.entity.TenantEntity;
 import org.apache.nifi.web.api.entity.UserEntity;
 import org.apache.nifi.web.api.entity.UserGroupEntity;
+import org.apache.nifi.web.api.entity.VariableEntity;
+import org.apache.nifi.web.api.entity.VariableRegistryEntity;
 import org.apache.nifi.web.controller.ControllerFacade;
 import org.apache.nifi.web.dao.AccessPolicyDAO;
 import org.apache.nifi.web.dao.ConnectionDAO;
@@ -217,28 +247,7 @@ import org.apache.nifi.web.util.SnippetUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.ws.rs.WebApplicationException;
-import javax.ws.rs.core.Response;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.ListIterator;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-import java.util.UUID;
-import java.util.function.Function;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
+import com.google.common.collect.Sets;
 
 /**
  * Implementation of NiFiServiceFacade that performs revision checking.
@@ -423,6 +432,11 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
     }
 
     @Override
+    public void verifyActivateControllerServices(final String groupId, final ControllerServiceState state, final Set<String> serviceIds) {
+        processGroupDAO.verifyActivateControllerServices(groupId, state, serviceIds);
+    }
+
+    @Override
     public void verifyDeleteProcessGroup(final String groupId) {
         processGroupDAO.verifyDelete(groupId);
     }
@@ -624,6 +638,10 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
      */
     private <D, C> RevisionUpdate<D> updateComponent(final Revision revision, final Authorizable authorizable, final Supplier<C> daoUpdate, final Function<C, D> dtoCreation) {
         final NiFiUser user = NiFiUserUtils.getNiFiUser();
+        return updateComponent(user, revision, authorizable, daoUpdate, dtoCreation);
+    }
+
+    private <D, C> RevisionUpdate<D> updateComponent(final NiFiUser user, final Revision revision, final Authorizable authorizable, final Supplier<C> daoUpdate, final Function<C, D> dtoCreation) {
         try {
             final RevisionUpdate<D> updatedComponent = revisionManager.updateRevision(new StandardRevisionClaim(revision), user, new UpdateRevisionTask<D>() {
                 @Override
@@ -774,6 +792,81 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
     }
 
     @Override
+    public Set<AffectedComponentDTO> getComponentsAffectedByVariableRegistryUpdate(final VariableRegistryDTO variableRegistryDto) {
+        final ProcessGroup group = processGroupDAO.getProcessGroup(variableRegistryDto.getProcessGroupId());
+        if (group == null) {
+            throw new ResourceNotFoundException("Could not find Process Group with ID " + variableRegistryDto.getProcessGroupId());
+        }
+
+        final Map<String, String> variableMap = new HashMap<>();
+        variableRegistryDto.getVariables().stream() // have to use forEach here instead of using Collectors.toMap because value may be null
+            .map(VariableEntity::getVariable)
+            .forEach(var -> variableMap.put(var.getName(), var.getValue()));
+
+        final Set<AffectedComponentDTO> affectedComponentDtos = new HashSet<>();
+
+        final Set<String> updatedVariableNames = getUpdatedVariables(group, variableMap);
+        for (final String variableName : updatedVariableNames) {
+            final Set<ConfiguredComponent> affectedComponents = group.getComponentsAffectedByVariable(variableName);
+
+            for (final ConfiguredComponent component : affectedComponents) {
+                if (component instanceof ProcessorNode) {
+                    final ProcessorNode procNode = (ProcessorNode) component;
+                    if (procNode.isRunning()) {
+                        affectedComponentDtos.add(dtoFactory.createAffectedComponentDto(procNode));
+                    }
+                } else if (component instanceof ControllerServiceNode) {
+                    final ControllerServiceNode serviceNode = (ControllerServiceNode) component;
+                    if (serviceNode.isActive()) {
+                        affectedComponentDtos.add(dtoFactory.createAffectedComponentDto(serviceNode));
+                    }
+                } else {
+                    throw new RuntimeException("Found unexpected type of Component [" + component.getCanonicalClassName() + "] dependending on variable");
+                }
+            }
+        }
+
+        return affectedComponentDtos;
+    }
+
+    private Set<String> getUpdatedVariables(final ProcessGroup group, final Map<String, String> newVariableValues) {
+        final Set<String> updatedVariableNames = new HashSet<>();
+
+        final ComponentVariableRegistry registry = group.getVariableRegistry();
+        for (final Map.Entry<String, String> entry : newVariableValues.entrySet()) {
+            final String varName = entry.getKey();
+            final String newValue = entry.getValue();
+
+            final String curValue = registry.getVariableValue(varName);
+            if (!Objects.equals(newValue, curValue)) {
+                updatedVariableNames.add(varName);
+            }
+        }
+
+        return updatedVariableNames;
+    }
+
+
+    @Override
+    public VariableRegistryEntity updateVariableRegistry(Revision revision, VariableRegistryDTO variableRegistryDto) {
+        return updateVariableRegistry(NiFiUserUtils.getNiFiUser(), revision, variableRegistryDto);
+    }
+
+    @Override
+    public VariableRegistryEntity updateVariableRegistry(NiFiUser user, Revision revision, VariableRegistryDTO variableRegistryDto) {
+        final ProcessGroup processGroupNode = processGroupDAO.getProcessGroup(variableRegistryDto.getProcessGroupId());
+        final RevisionUpdate<VariableRegistryDTO> snapshot = updateComponent(user, revision,
+            processGroupNode,
+            () -> processGroupDAO.updateVariableRegistry(variableRegistryDto),
+            processGroup -> dtoFactory.createVariableRegistryDto(processGroup));
+
+        final PermissionsDTO permissions = dtoFactory.createPermissionsDto(processGroupNode);
+        final RevisionDTO updatedRevision = dtoFactory.createRevisionDTO(snapshot.getLastModification());
+        return entityFactory.createVariableRegistryEntity(snapshot.getComponent(), updatedRevision, permissions);
+    }
+
+
+    @Override
     public ProcessGroupEntity updateProcessGroup(final Revision revision, final ProcessGroupDTO processGroupDTO) {
         final ProcessGroup processGroupNode = processGroupDAO.getProcessGroup(processGroupDTO.getId());
         final RevisionUpdate<ProcessGroupDTO> snapshot = updateComponent(revision,
@@ -790,14 +883,27 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
     }
 
     @Override
+    public void verifyUpdateProcessGroup(ProcessGroupDTO processGroupDTO) {
+        if (processGroupDAO.hasProcessGroup(processGroupDTO.getId())) {
+            processGroupDAO.verifyUpdate(processGroupDTO);
+        }
+    }
+
+    @Override
     public ScheduleComponentsEntity scheduleComponents(final String processGroupId, final ScheduledState state, final Map<String, Revision> componentRevisions) {
         final NiFiUser user = NiFiUserUtils.getNiFiUser();
+        return scheduleComponents(user, processGroupId, state, componentRevisions);
+    }
+
+    @Override
+    public ScheduleComponentsEntity scheduleComponents(final NiFiUser user, final String processGroupId, final ScheduledState state, final Map<String, Revision> componentRevisions) {
+
         final RevisionUpdate<ScheduleComponentsEntity> updatedComponent = revisionManager.updateRevision(new StandardRevisionClaim(componentRevisions.values()), user, new
                 UpdateRevisionTask<ScheduleComponentsEntity>() {
                     @Override
                     public RevisionUpdate<ScheduleComponentsEntity> update() {
                         // schedule the components
-                        processGroupDAO.scheduleComponents(processGroupId, state, componentRevisions.keySet());
+                processGroupDAO.scheduleComponents(processGroupId, state, componentRevisions.keySet());
 
                         // update the revisions
                         final Map<String, Revision> updatedRevisions = new HashMap<>();
@@ -821,6 +927,46 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
     }
 
     @Override
+    public ActivateControllerServicesEntity activateControllerServices(final String processGroupId, final ControllerServiceState state, final Map<String, Revision> serviceRevisions) {
+
+        final NiFiUser user = NiFiUserUtils.getNiFiUser();
+        return activateControllerServices(user, processGroupId, state, serviceRevisions);
+    }
+
+    @Override
+    public ActivateControllerServicesEntity activateControllerServices(final NiFiUser user, final String processGroupId, final ControllerServiceState state,
+        final Map<String, Revision> serviceRevisions) {
+
+        final RevisionUpdate<ActivateControllerServicesEntity> updatedComponent = revisionManager.updateRevision(new StandardRevisionClaim(serviceRevisions.values()), user,
+            new UpdateRevisionTask<ActivateControllerServicesEntity>() {
+                @Override
+                public RevisionUpdate<ActivateControllerServicesEntity> update() {
+                    // schedule the components
+                    processGroupDAO.activateControllerServices(processGroupId, state, serviceRevisions.keySet());
+
+                    // update the revisions
+                    final Map<String, Revision> updatedRevisions = new HashMap<>();
+                    for (final Revision revision : serviceRevisions.values()) {
+                        final Revision currentRevision = revisionManager.getRevision(revision.getComponentId());
+                        updatedRevisions.put(revision.getComponentId(), currentRevision.incrementRevision(revision.getClientId()));
+                    }
+
+                    // save
+                    controllerFacade.save();
+
+                    // gather details for response
+                    final ActivateControllerServicesEntity entity = new ActivateControllerServicesEntity();
+                    entity.setId(processGroupId);
+                    entity.setState(state.name());
+                    return new StandardRevisionUpdate<>(entity, null, new HashSet<>(updatedRevisions.values()));
+                }
+            });
+
+        return updatedComponent.getComponent();
+    }
+
+
+    @Override
     public ControllerConfigurationEntity updateControllerConfiguration(final Revision revision, final ControllerConfigurationDTO controllerConfigurationDTO) {
         final RevisionUpdate<ControllerConfigurationDTO> updatedComponent = updateComponent(
                 revision,
@@ -3062,7 +3208,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
         return createProcessGroupEntity(processGroup);
     }
 
-    private ControllerServiceEntity createControllerServiceEntity(final ControllerServiceNode serviceNode, final Set<String> serviceIds) {
+    private ControllerServiceEntity createControllerServiceEntity(final ControllerServiceNode serviceNode, final Set<String> serviceIds, final NiFiUser user) {
         final ControllerServiceDTO dto = dtoFactory.createControllerServiceDto(serviceNode);
 
         final ControllerServiceReference ref = serviceNode.getReferences();
@@ -3070,26 +3216,77 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
         dto.setReferencingComponents(referencingComponentsEntity.getControllerServiceReferencingComponents());
 
         final RevisionDTO revision = dtoFactory.createRevisionDTO(revisionManager.getRevision(serviceNode.getIdentifier()));
-        final PermissionsDTO permissions = dtoFactory.createPermissionsDto(serviceNode);
+        final PermissionsDTO permissions = dtoFactory.createPermissionsDto(serviceNode, user);
         final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(serviceNode.getIdentifier()));
         final List<BulletinEntity> bulletinEntities = bulletins.stream().map(bulletin -> entityFactory.createBulletinEntity(bulletin, permissions.getCanRead())).collect(Collectors.toList());
         return entityFactory.createControllerServiceEntity(dto, revision, permissions, bulletinEntities);
     }
 
     @Override
-    public Set<ControllerServiceEntity> getControllerServices(final String groupId) {
-        final Set<ControllerServiceNode> serviceNodes = controllerServiceDAO.getControllerServices(groupId);
+    public VariableRegistryEntity getVariableRegistry(final String groupId, final boolean includeAncestorGroups) {
+        final ProcessGroup processGroup = processGroupDAO.getProcessGroup(groupId);
+        if (processGroup == null) {
+            throw new ResourceNotFoundException("Could not find group with ID " + groupId);
+        }
+
+        return createVariableRegistryEntity(processGroup, includeAncestorGroups);
+    }
+
+    private VariableRegistryEntity createVariableRegistryEntity(final ProcessGroup processGroup, final boolean includeAncestorGroups) {
+        final VariableRegistryDTO registryDto = dtoFactory.createVariableRegistryDto(processGroup);
+        final RevisionDTO revision = dtoFactory.createRevisionDTO(revisionManager.getRevision(processGroup.getIdentifier()));
+        final PermissionsDTO permissions = dtoFactory.createPermissionsDto(processGroup);
+
+        if (includeAncestorGroups) {
+            ProcessGroup parent = processGroup.getParent();
+            while (parent != null) {
+                final PermissionsDTO parentPerms = dtoFactory.createPermissionsDto(processGroup);
+                if (Boolean.TRUE.equals(parentPerms.getCanRead())) {
+                    final VariableRegistryDTO parentRegistryDto = dtoFactory.createVariableRegistryDto(parent);
+                    final Set<VariableEntity> parentVariables = parentRegistryDto.getVariables();
+                    registryDto.getVariables().addAll(parentVariables);
+                }
+
+                parent = parent.getParent();
+            }
+        }
+
+        return entityFactory.createVariableRegistryEntity(registryDto, revision, permissions);
+    }
+
+    @Override
+    public VariableRegistryEntity populateAffectedComponents(final VariableRegistryDTO variableRegistryDto) {
+        final String groupId = variableRegistryDto.getProcessGroupId();
+        final ProcessGroup processGroup = processGroupDAO.getProcessGroup(groupId);
+        if (processGroup == null) {
+            throw new ResourceNotFoundException("Could not find group with ID " + groupId);
+        }
+
+        final VariableRegistryDTO registryDto = dtoFactory.populateAffectedComponents(variableRegistryDto, processGroup);
+        final RevisionDTO revision = dtoFactory.createRevisionDTO(revisionManager.getRevision(processGroup.getIdentifier()));
+        final PermissionsDTO permissions = dtoFactory.createPermissionsDto(processGroup);
+        return entityFactory.createVariableRegistryEntity(registryDto, revision, permissions);
+    }
+
+    @Override
+    public Set<ControllerServiceEntity> getControllerServices(final String groupId, final boolean includeAncestorGroups, final boolean includeDescendantGroups) {
+        return getControllerServices(groupId, includeAncestorGroups, includeDescendantGroups, NiFiUserUtils.getNiFiUser());
+    }
+
+    @Override
+    public Set<ControllerServiceEntity> getControllerServices(final String groupId, final boolean includeAncestorGroups, final boolean includeDescendantGroups, final NiFiUser user) {
+        final Set<ControllerServiceNode> serviceNodes = controllerServiceDAO.getControllerServices(groupId, includeAncestorGroups, includeDescendantGroups);
         final Set<String> serviceIds = serviceNodes.stream().map(service -> service.getIdentifier()).collect(Collectors.toSet());
 
         return serviceNodes.stream()
-            .map(serviceNode -> createControllerServiceEntity(serviceNode, serviceIds))
+            .map(serviceNode -> createControllerServiceEntity(serviceNode, serviceIds, user))
             .collect(Collectors.toSet());
     }
 
     @Override
     public ControllerServiceEntity getControllerService(final String controllerServiceId) {
         final ControllerServiceNode controllerService = controllerServiceDAO.getControllerService(controllerServiceId);
-        return createControllerServiceEntity(controllerService, Sets.newHashSet(controllerServiceId));
+        return createControllerServiceEntity(controllerService, Sets.newHashSet(controllerServiceId), NiFiUserUtils.getNiFiUser());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
index 3d78741..1a50d04 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
@@ -608,6 +608,11 @@ public abstract class ApplicationResource {
             serviceFacade.authorizeAccess(authorizer);
             serviceFacade.verifyRevision(revision, user);
 
+            // verify if necessary
+            if (verifier != null) {
+                verifier.run();
+            }
+
             return action.apply(revision, entity);
         }
     }
@@ -657,6 +662,11 @@ public abstract class ApplicationResource {
             serviceFacade.authorizeAccess(authorizer);
             serviceFacade.verifyRevisions(revisions, user);
 
+            // verify if necessary
+            if (verifier != null) {
+                verifier.run();
+            }
+
             return action.apply(revisions, entity);
         }
     }
@@ -820,16 +830,16 @@ public abstract class ApplicationResource {
         }
     }
 
-    /**
-     * Replicates the request to the given node
-     *
-     * @param method   the HTTP method
-     * @param nodeUuid the UUID of the node to replicate the request to
-     * @return the response from the node
-     * @throws UnknownNodeException if the nodeUuid given does not map to any node in the cluster
-     */
-    protected Response replicate(final String method, final String nodeUuid) {
-        return replicate(method, getRequestParameters(), nodeUuid);
+
+    private void ensureFlowInitialized() {
+        if (!flowController.isInitialized()) {
+            throw new IllegalClusterStateException("Cluster is still in the process of voting on the appropriate Data Flow.");
+        }
+    }
+
+    protected Response replicate(final String method, final Object entity, final String nodeUuid, final Map<String, String> headersToOverride) {
+        final URI path = getAbsolutePath();
+        return replicate(path, method, entity, nodeUuid, headersToOverride);
     }
 
     /**
@@ -845,22 +855,16 @@ public abstract class ApplicationResource {
         return replicate(method, entity, nodeUuid, null);
     }
 
-    private void ensureFlowInitialized() {
-        if (!flowController.isInitialized()) {
-            throw new IllegalClusterStateException("Cluster is still in the process of voting on the appropriate Data Flow.");
-        }
-    }
-
     /**
      * Replicates the request to the given node
      *
-     * @param method   the HTTP method
-     * @param entity   the Entity to replicate
+     * @param method the HTTP method
+     * @param entity the Entity to replicate
      * @param nodeUuid the UUID of the node to replicate the request to
      * @return the response from the node
      * @throws UnknownNodeException if the nodeUuid given does not map to any node in the cluster
      */
-    protected Response replicate(final String method, final Object entity, final String nodeUuid, final Map<String, String> headersToOverride) {
+    protected Response replicate(final URI path, final String method, final Object entity, final String nodeUuid, final Map<String, String> headersToOverride) {
         // since we're cluster we must specify the cluster node identifier
         if (nodeUuid == null) {
             throw new IllegalArgumentException("The cluster node identifier must be specified.");
@@ -873,7 +877,6 @@ public abstract class ApplicationResource {
 
         ensureFlowInitialized();
 
-        final URI path = getAbsolutePath();
         try {
             final Map<String, String> headers = headersToOverride == null ? getHeaders() : getHeaders(headersToOverride);
 
@@ -996,6 +999,12 @@ public abstract class ApplicationResource {
         }
     }
 
+
+    protected NodeResponse replicateNodeResponse(final String method, final Object entity, final Map<String, String> headersToOverride) throws InterruptedException {
+        final URI path = getAbsolutePath();
+        return replicateNodeResponse(path, method, entity, headersToOverride);
+    }
+
     /**
      * Replicates the request to all nodes in the cluster using the provided method and entity. The headers
      * used will be those provided by the {@link #getHeaders()} method. The URI that will be used will be
@@ -1009,10 +1018,9 @@ public abstract class ApplicationResource {
      * @throws InterruptedException if interrupted while replicating the request
      * @see #replicate(String, Object, Map)
      */
-    protected NodeResponse replicateNodeResponse(final String method, final Object entity, final Map<String, String> headersToOverride) throws InterruptedException {
+    protected NodeResponse replicateNodeResponse(final URI path, final String method, final Object entity, final Map<String, String> headersToOverride) throws InterruptedException {
         ensureFlowInitialized();
 
-        final URI path = getAbsolutePath();
         final Map<String, String> headers = headersToOverride == null ? getHeaders() : getHeaders(headersToOverride);
 
         // Determine whether we should replicate only to the cluster coordinator, or if we should replicate directly

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
index 889676c..5d5a796 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
@@ -36,6 +36,8 @@ import org.apache.nifi.cluster.coordination.node.NodeConnectionState;
 import org.apache.nifi.cluster.manager.NodeResponse;
 import org.apache.nifi.cluster.protocol.NodeIdentifier;
 import org.apache.nifi.controller.ScheduledState;
+import org.apache.nifi.controller.service.ControllerServiceNode;
+import org.apache.nifi.controller.service.ControllerServiceState;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.nar.NarClassLoaders;
 import org.apache.nifi.util.NiFiProperties;
@@ -61,6 +63,7 @@ import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
 import org.apache.nifi.web.api.dto.status.ControllerStatusDTO;
 import org.apache.nifi.web.api.entity.AboutEntity;
 import org.apache.nifi.web.api.entity.ActionEntity;
+import org.apache.nifi.web.api.entity.ActivateControllerServicesEntity;
 import org.apache.nifi.web.api.entity.BannerEntity;
 import org.apache.nifi.web.api.entity.BulletinBoardEntity;
 import org.apache.nifi.web.api.entity.ClusteSummaryEntity;
@@ -118,6 +121,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.function.Predicate;
 import java.util.stream.Collectors;
 
 /**
@@ -388,7 +392,7 @@ public class FlowResource extends ApplicationResource {
         }
 
         // get all the controller services
-        final Set<ControllerServiceEntity> controllerServices = serviceFacade.getControllerServices(null);
+        final Set<ControllerServiceEntity> controllerServices = serviceFacade.getControllerServices(null, false, false);
         controllerServiceResource.populateRemainingControllerServiceEntitiesContent(controllerServices);
 
         // create the response entity
@@ -426,11 +430,10 @@ public class FlowResource extends ApplicationResource {
             }
     )
     public Response getControllerServicesFromGroup(
-            @ApiParam(
-                    value = "The process group id.",
-                    required = true
-            )
-            @PathParam("id") String groupId) throws InterruptedException {
+            @ApiParam(value = "The process group id.", required = true) @PathParam("id") String groupId,
+            @ApiParam("Whether or not to include parent/ancestory process groups") @QueryParam("includeAncestorGroups") @DefaultValue("true") boolean includeAncestorGroups,
+            @ApiParam("Whether or not to include descendant process groups") @QueryParam("includeDescendantGroups") @DefaultValue("false") boolean includeDescendantGroups
+            ) throws InterruptedException {
 
         authorizeFlow();
 
@@ -439,7 +442,7 @@ public class FlowResource extends ApplicationResource {
         }
 
         // get all the controller services
-        final Set<ControllerServiceEntity> controllerServices = serviceFacade.getControllerServices(groupId);
+        final Set<ControllerServiceEntity> controllerServices = serviceFacade.getControllerServices(groupId, includeAncestorGroups, includeDescendantGroups);
         controllerServiceResource.populateRemainingControllerServiceEntitiesContent(controllerServices);
 
         // create the response entity
@@ -512,7 +515,7 @@ public class FlowResource extends ApplicationResource {
     @Produces(MediaType.APPLICATION_JSON)
     @Path("process-groups/{id}")
     @ApiOperation(
-            value = "Schedule or unschedule comopnents in the specified Process Group.",
+            value = "Schedule or unschedule components in the specified Process Group.",
             response = ScheduleComponentsEntity.class,
             authorizations = {
                     @Authorization(value = "Read - /flow", type = ""),
@@ -570,7 +573,7 @@ public class FlowResource extends ApplicationResource {
 
                 // ensure authorized for each processor we will attempt to schedule
                 group.findAllProcessors().stream()
-                        .filter(ScheduledState.RUNNING.equals(state) ? ProcessGroup.SCHEDULABLE_PROCESSORS : ProcessGroup.UNSCHEDULABLE_PROCESSORS)
+                    .filter(ScheduledState.RUNNING.equals(state) ? ProcessGroup.SCHEDULABLE_PROCESSORS : ProcessGroup.UNSCHEDULABLE_PROCESSORS)
                         .filter(processor -> processor.isAuthorized(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()))
                         .forEach(processor -> {
                             componentIds.add(processor.getIdentifier());
@@ -578,7 +581,7 @@ public class FlowResource extends ApplicationResource {
 
                 // ensure authorized for each input port we will attempt to schedule
                 group.findAllInputPorts().stream()
-                        .filter(ScheduledState.RUNNING.equals(state) ? ProcessGroup.SCHEDULABLE_PORTS : ProcessGroup.UNSCHEDULABLE_PORTS)
+                    .filter(ScheduledState.RUNNING.equals(state) ? ProcessGroup.SCHEDULABLE_PORTS : ProcessGroup.UNSCHEDULABLE_PORTS)
                         .filter(inputPort -> inputPort.isAuthorized(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()))
                         .forEach(inputPort -> {
                             componentIds.add(inputPort.getIdentifier());
@@ -586,7 +589,7 @@ public class FlowResource extends ApplicationResource {
 
                 // ensure authorized for each output port we will attempt to schedule
                 group.findAllOutputPorts().stream()
-                        .filter(ScheduledState.RUNNING.equals(state) ? ProcessGroup.SCHEDULABLE_PORTS : ProcessGroup.UNSCHEDULABLE_PORTS)
+                    .filter(ScheduledState.RUNNING.equals(state) ? ProcessGroup.SCHEDULABLE_PORTS : ProcessGroup.UNSCHEDULABLE_PORTS)
                         .filter(outputPort -> outputPort.isAuthorized(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()))
                         .forEach(outputPort -> {
                             componentIds.add(outputPort.getIdentifier());
@@ -640,7 +643,129 @@ public class FlowResource extends ApplicationResource {
                             componentsToSchedule.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> getRevision(e.getValue(), e.getKey())));
 
                     // update the process group
-                    final ScheduleComponentsEntity entity = serviceFacade.scheduleComponents(id, scheduledState, componentRevisions);
+                final ScheduleComponentsEntity entity = serviceFacade.scheduleComponents(id, scheduledState, componentRevisions);
+                    return generateOkResponse(entity).build();
+                }
+        );
+    }
+
+
+    @PUT
+    @Consumes(MediaType.APPLICATION_JSON)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("process-groups/{id}/controller-services")
+    @ApiOperation(value = "Enable or disable Controller Services in the specified Process Group.",
+        response = ActivateControllerServicesEntity.class,
+        authorizations = {
+            @Authorization(value = "Read - /flow", type = ""),
+            @Authorization(value = "Write - /{component-type}/{uuid} - For every service being enabled/disabled", type = "")
+        })
+    @ApiResponses(
+            value = {
+                    @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+                    @ApiResponse(code = 401, message = "Client could not be authenticated."),
+                    @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+                    @ApiResponse(code = 404, message = "The specified resource could not be found."),
+                    @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+            }
+    )
+    public Response activateControllerServices(
+            @Context HttpServletRequest httpServletRequest,
+            @ApiParam(value = "The process group id.", required = true)
+            @PathParam("id") String id,
+            @ApiParam(value = "The request to schedule or unschedule. If the comopnents in the request are not specified, all authorized components will be considered.", required = true)
+            final ActivateControllerServicesEntity requestEntity) {
+
+        // ensure the same id is being used
+        if (!id.equals(requestEntity.getId())) {
+            throw new IllegalArgumentException(String.format("The process group id (%s) in the request body does "
+                + "not equal the process group id of the requested resource (%s).", requestEntity.getId(), id));
+        }
+
+        final ControllerServiceState state;
+        if (requestEntity.getState() == null) {
+            throw new IllegalArgumentException("The controller service state must be specified.");
+        } else {
+            try {
+                state = ControllerServiceState.valueOf(requestEntity.getState());
+            } catch (final IllegalArgumentException iae) {
+                throw new IllegalArgumentException(String.format("The controller service state must be one of [%s].",
+                    StringUtils.join(EnumSet.of(ControllerServiceState.ENABLED, ControllerServiceState.DISABLED), ", ")));
+            }
+        }
+
+        // ensure its a supported scheduled state
+        if (ControllerServiceState.DISABLING.equals(state) || ControllerServiceState.ENABLING.equals(state)) {
+            throw new IllegalArgumentException(String.format("The scheduled must be one of [%s].",
+                StringUtils.join(EnumSet.of(ControllerServiceState.ENABLED, ControllerServiceState.DISABLED), ", ")));
+        }
+
+        // if the components are not specified, gather all components and their current revision
+        if (requestEntity.getComponents() == null) {
+            // get the current revisions for the components being updated
+            final Set<Revision> revisions = serviceFacade.getRevisionsFromGroup(id, group -> {
+                final Set<String> componentIds = new HashSet<>();
+
+                final Predicate<ControllerServiceNode> filter;
+                if (ControllerServiceState.ENABLED.equals(state)) {
+                    filter = service -> !service.isActive() && service.isValid();
+                } else {
+                    filter = service -> service.isActive();
+                }
+
+                group.findAllControllerServices().stream()
+                    .filter(filter)
+                    .filter(service -> service.isAuthorized(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()))
+                    .forEach(service -> componentIds.add(service.getIdentifier()));
+                return componentIds;
+            });
+
+            // build the component mapping
+            final Map<String, RevisionDTO> componentsToSchedule = new HashMap<>();
+            revisions.forEach(revision -> {
+                final RevisionDTO dto = new RevisionDTO();
+                dto.setClientId(revision.getClientId());
+                dto.setVersion(revision.getVersion());
+                componentsToSchedule.put(revision.getComponentId(), dto);
+            });
+
+            // set the components and their current revision
+            requestEntity.setComponents(componentsToSchedule);
+        }
+
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.PUT, requestEntity);
+        }
+
+        final Map<String, RevisionDTO> requestComponentsToSchedule = requestEntity.getComponents();
+        final Map<String, Revision> requestComponentRevisions =
+                requestComponentsToSchedule.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> getRevision(e.getValue(), e.getKey())));
+        final Set<Revision> requestRevisions = new HashSet<>(requestComponentRevisions.values());
+
+        return withWriteLock(
+                serviceFacade,
+                requestEntity,
+                requestRevisions,
+                lookup -> {
+                    // ensure access to the flow
+                    authorizeFlow();
+
+                    // ensure access to every component being scheduled
+                    requestComponentsToSchedule.keySet().forEach(componentId -> {
+                        final Authorizable authorizable = lookup.getControllerService(componentId).getAuthorizable();
+                        authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+                    });
+                },
+                () -> serviceFacade.verifyActivateControllerServices(id, state, requestComponentRevisions.keySet()),
+                (revisions, scheduleComponentsEntity) -> {
+                final ControllerServiceState serviceState = ControllerServiceState.valueOf(scheduleComponentsEntity.getState());
+
+                    final Map<String, RevisionDTO> componentsToSchedule = scheduleComponentsEntity.getComponents();
+                    final Map<String, Revision> componentRevisions =
+                            componentsToSchedule.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> getRevision(e.getValue(), e.getKey())));
+
+                    // update the controller services
+                final ActivateControllerServicesEntity entity = serviceFacade.activateControllerServices(id, serviceState, componentRevisions);
                     return generateOkResponse(entity).build();
                 }
         );


[9/9] nifi git commit: NIFI-4224: - Initial implementation of Process Group level Variable Registry - Updated to incorporate PR Feedback - Changed log message because slf4j-simple apparently has a memory leak; passing a String instead of passing in the C

Posted by mc...@apache.org.
NIFI-4224:
- Initial implementation of Process Group level Variable Registry
- Updated to incorporate PR Feedback
- Changed log message because slf4j-simple apparently has a memory leak; passing a String instead of passing in the Controller Service object as an argument addresses this.
- This closes #2051


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

Branch: refs/heads/master
Commit: 5cd8e93beb30075ebe8623c2a278a5c9cf1afc75
Parents: c1b99d5
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Jul 25 13:46:10 2017 -0400
Committer: Matt Gilman <ma...@gmail.com>
Committed: Thu Aug 17 10:42:29 2017 -0400

----------------------------------------------------------------------
 .../registry/ComponentVariableRegistry.java     |   24 +
 .../nifi/registry/VariableDescriptor.java       |   77 +-
 .../expression/language/EmptyPreparedQuery.java |    6 +
 .../language/InvalidPreparedQuery.java          |    6 +
 .../language/NamedVariableImpact.java           |   34 +
 .../language/NoVariablesImpacted.java           |   25 +
 .../expression/language/PreparedQuery.java      |   10 +
 .../attribute/expression/language/Query.java    | 1082 +----------------
 .../language/StandardPreparedQuery.java         |   85 +-
 .../expression/language/ValueLookup.java        |    1 +
 .../expression/language/VariableImpact.java     |   26 +
 .../language/compile/CompiledExpression.java    |   53 +
 .../language/compile/ExpressionCompiler.java    | 1122 ++++++++++++++++++
 .../functions/AttributeEvaluator.java           |   45 -
 .../selection/AllAttributesEvaluator.java       |    4 +
 .../selection/AnyAttributeEvaluator.java        |    4 +
 .../selection/AttributeEvaluator.java           |   49 +
 .../evaluation/selection/MappingEvaluator.java  |    4 +
 .../selection/MultiNamedAttributeEvaluator.java |    4 +
 .../expression/language/TestQuery.java          |   44 +-
 .../language/TestStandardPreparedQuery.java     |   41 +
 .../expression/language/TestValueLookup.java    |    4 +-
 .../nifi/web/api/dto/AffectedComponentDTO.java  |   59 +
 .../nifi/web/api/dto/ProcessGroupDTO.java       |   16 +
 .../apache/nifi/web/api/dto/VariableDTO.java    |   69 ++
 .../nifi/web/api/dto/VariableRegistryDTO.java   |   50 +
 .../dto/VariableRegistryUpdateRequestDTO.java   |  115 ++
 .../api/dto/VariableRegistryUpdateStepDTO.java  |   59 +
 .../ActivateControllerServicesEntity.java       |   66 ++
 .../api/entity/ScheduleComponentsEntity.java    |   12 +-
 .../nifi/web/api/entity/VariableEntity.java     |   51 +
 .../web/api/entity/VariableRegistryEntity.java  |   50 +
 .../VariableRegistryUpdateRequestEntity.java    |   49 +
 .../http/StandardHttpResponseMapper.java        |   19 +-
 .../controller/AbstractConfiguredComponent.java |   42 +-
 .../nifi/controller/ConfiguredComponent.java    |    7 +
 .../nifi/controller/ProcessScheduler.java       |    9 +-
 .../apache/nifi/controller/ProcessorNode.java   |   19 +-
 .../service/ControllerServiceNode.java          |    2 +-
 .../service/ControllerServiceProvider.java      |    6 +-
 .../service/ControllerServiceReference.java     |   10 +
 .../org/apache/nifi/groups/ProcessGroup.java    |   51 +-
 .../apache/nifi/controller/FlowController.java  |   46 +-
 .../controller/StandardFlowSynchronizer.java    |   70 +-
 .../nifi/controller/StandardProcessorNode.java  |   77 +-
 .../reporting/AbstractReportingTaskNode.java    |   22 +-
 .../reporting/StandardReportingTaskNode.java    |    8 +-
 .../scheduling/EventDrivenSchedulingAgent.java  |    9 +-
 .../scheduling/QuartzSchedulingAgent.java       |    8 +-
 .../scheduling/StandardProcessScheduler.java    |   42 +-
 .../scheduling/TimerDrivenSchedulingAgent.java  |    6 +-
 .../serialization/FlowFromDOMFactory.java       |   10 +
 .../serialization/StandardFlowSerializer.java   |   16 +-
 .../service/ServiceStateTransition.java         |   68 ++
 .../service/StandardControllerServiceNode.java  |   71 +-
 .../StandardControllerServiceProvider.java      |   76 +-
 .../StandardControllerServiceReference.java     |   33 +
 .../nifi/fingerprint/FingerprintFactory.java    |   69 +-
 .../nifi/groups/StandardProcessGroup.java       |  189 ++-
 .../nifi/processor/StandardProcessContext.java  |   12 +-
 .../processor/StandardValidationContext.java    |    7 +-
 .../variable/FileBasedVariableRegistry.java     |   89 ++
 .../variable/MutableVariableRegistry.java       |   52 +
 .../StandardComponentVariableRegistry.java      |   94 ++
 .../nifi/util/FileBasedVariableRegistry.java    |   88 --
 .../src/main/resources/FlowConfiguration.xsd    |    6 +
 .../src/main/resources/nifi-context.xml         |    2 +-
 .../controller/StandardFlowServiceTest.java     |    2 +-
 .../nifi/controller/TestFlowController.java     |    2 +-
 .../controller/TestStandardProcessorNode.java   |   48 +-
 .../reporting/TestStandardReportingContext.java |   20 +-
 .../scheduling/TestProcessorLifecycle.java      |    2 +-
 .../TestStandardProcessScheduler.java           |   10 +-
 .../StandardFlowSerializerTest.java             |    2 +-
 .../StandardControllerServiceProviderTest.java  |    2 +-
 .../TestStandardControllerServiceProvider.java  |   41 +-
 .../service/mock/MockProcessGroup.java          |   32 +-
 .../util/TestFileBasedVariableRegistry.java     |    1 +
 .../apache/nifi/audit/ProcessGroupAuditor.java  |   63 +-
 .../variable/VariableRegistryUpdateRequest.java |  109 ++
 .../variable/VariableRegistryUpdateStep.java    |   48 +
 .../org/apache/nifi/web/NiFiServiceFacade.java  |  126 +-
 .../apache/nifi/web/NiFiServiceFacadeLock.java  |   12 +
 .../nifi/web/StandardNiFiServiceFacade.java     |  257 +++-
 .../nifi/web/api/ApplicationResource.java       |   52 +-
 .../org/apache/nifi/web/api/FlowResource.java   |  149 ++-
 .../nifi/web/api/ProcessGroupResource.java      | 1034 +++++++++++++++-
 .../org/apache/nifi/web/api/dto/DtoFactory.java |  217 +++-
 .../apache/nifi/web/api/dto/EntityFactory.java  |   13 +
 .../nifi/web/dao/ControllerServiceDAO.java      |    2 +-
 .../apache/nifi/web/dao/ProcessGroupDAO.java    |   44 +-
 .../dao/impl/StandardControllerServiceDAO.java  |    9 +-
 .../web/dao/impl/StandardProcessGroupDAO.java   |   80 +-
 .../java/org/apache/nifi/web/util/Pause.java    |   30 +
 .../src/main/resources/nifi-web-api-context.xml |    1 +
 95 files changed, 5258 insertions(+), 1831 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-api/src/main/java/org/apache/nifi/registry/ComponentVariableRegistry.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/registry/ComponentVariableRegistry.java b/nifi-api/src/main/java/org/apache/nifi/registry/ComponentVariableRegistry.java
new file mode 100644
index 0000000..7284b77
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/registry/ComponentVariableRegistry.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.registry;
+
+public interface ComponentVariableRegistry extends VariableRegistry {
+    VariableRegistry getParent();
+
+    void setParent(VariableRegistry parentRegistry);
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-api/src/main/java/org/apache/nifi/registry/VariableDescriptor.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/registry/VariableDescriptor.java b/nifi-api/src/main/java/org/apache/nifi/registry/VariableDescriptor.java
index 5b1e88f..f9e9ce2 100644
--- a/nifi-api/src/main/java/org/apache/nifi/registry/VariableDescriptor.java
+++ b/nifi-api/src/main/java/org/apache/nifi/registry/VariableDescriptor.java
@@ -66,6 +66,44 @@ public final class VariableDescriptor implements Comparable<VariableDescriptor>
         return getName().compareTo(o.getName());
     }
 
+    public String getName() {
+        return name;
+    }
+
+    public String getDescription() {
+        return description;
+    }
+
+    public boolean isSensitive() {
+        return sensitive;
+    }
+
+    @Override
+    public boolean equals(final Object other) {
+        if (other == null) {
+            return false;
+        }
+        if (this == other) {
+            return true;
+        }
+        if (!(other instanceof VariableDescriptor)) {
+            return false;
+        }
+
+        final VariableDescriptor desc = (VariableDescriptor) other;
+        return this.name.equals(desc.name);
+    }
+
+    @Override
+    public int hashCode() {
+        return 797 + this.name.hashCode() * 97;
+    }
+
+    @Override
+    public String toString() {
+        return getClass().getSimpleName() + "[" + name + "]";
+    }
+
     public static final class Builder {
 
         private String name = null;
@@ -112,43 +150,4 @@ public final class VariableDescriptor implements Comparable<VariableDescriptor>
             return new VariableDescriptor(this);
         }
     }
-
-    public String getName() {
-        return name;
-    }
-
-    public String getDescription() {
-        return description;
-    }
-
-    public boolean isSensitive() {
-        return sensitive;
-    }
-
-    @Override
-    public boolean equals(final Object other) {
-        if (other == null) {
-            return false;
-        }
-        if (!(other instanceof VariableDescriptor)) {
-            return false;
-        }
-        if (this == other) {
-            return true;
-        }
-
-        final VariableDescriptor desc = (VariableDescriptor) other;
-        return this.name.equals(desc.name);
-    }
-
-    @Override
-    public int hashCode() {
-        return 797 + this.name.hashCode() * 97;
-    }
-
-    @Override
-    public String toString() {
-        return getClass().getSimpleName() + "[" + name + "]";
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java
index a435b08..e9ac03b 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language;
 
 
 import java.util.Map;
+
 import org.apache.nifi.expression.AttributeValueDecorator;
 import org.apache.nifi.processor.exception.ProcessException;
 
@@ -43,4 +44,9 @@ public class EmptyPreparedQuery implements PreparedQuery {
     public boolean isExpressionLanguagePresent() {
         return false;
     }
+
+    @Override
+    public VariableImpact getVariableImpact() {
+        return VariableImpact.NEVER_IMPACTED;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java
index ce0dec3..8ef996b 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language;
 
 
 import java.util.Map;
+
 import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageException;
 import org.apache.nifi.expression.AttributeValueDecorator;
 import org.apache.nifi.processor.exception.ProcessException;
@@ -52,4 +53,9 @@ public class InvalidPreparedQuery implements PreparedQuery {
     public boolean isExpressionLanguagePresent() {
         return false;
     }
+
+    @Override
+    public VariableImpact getVariableImpact() {
+        return VariableImpact.NEVER_IMPACTED;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/NamedVariableImpact.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/NamedVariableImpact.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/NamedVariableImpact.java
new file mode 100644
index 0000000..730c6ec
--- /dev/null
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/NamedVariableImpact.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.attribute.expression.language;
+
+import java.util.Set;
+
+public class NamedVariableImpact implements VariableImpact {
+    private final Set<String> impactedVariableNames;
+
+    public NamedVariableImpact(final Set<String> impactedVariableNames) {
+        this.impactedVariableNames = impactedVariableNames;
+    }
+
+    @Override
+    public boolean isImpacted(final String variableName) {
+        return impactedVariableNames.contains(variableName);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/NoVariablesImpacted.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/NoVariablesImpacted.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/NoVariablesImpacted.java
new file mode 100644
index 0000000..669a532
--- /dev/null
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/NoVariablesImpacted.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.attribute.expression.language;
+
+public class NoVariablesImpacted implements VariableImpact {
+    @Override
+    public boolean isImpacted(final String variableName) {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java
index 5552cac..c51656d 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language;
 
 
 import java.util.Map;
+
 import org.apache.nifi.expression.AttributeValueDecorator;
 import org.apache.nifi.processor.exception.ProcessException;
 
@@ -28,4 +29,13 @@ public interface PreparedQuery {
     String evaluateExpressions(final Map<String, String> valueLookup, final AttributeValueDecorator decorator, final Map<String, String> stateVariables) throws ProcessException;
 
     boolean isExpressionLanguagePresent();
+
+    /**
+     * Returns a {@link VariableImpact} that can be used to determine whether or not a given
+     * variable impacts this Expression.
+     *
+     * @return a {@link VariableImpact} that can be used to determine whether or not a given
+     *         variable impacts this Expression.
+     */
+    VariableImpact getVariableImpact();
 }


[5/9] nifi git commit: NIFI-4224: - Initial implementation of Process Group level Variable Registry - Updated to incorporate PR Feedback - Changed log message because slf4j-simple apparently has a memory leak; passing a String instead of passing in the C

Posted by mc...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
index 9611b73..07923c6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
@@ -16,6 +16,14 @@
  */
 package org.apache.nifi.controller.reporting;
 
+import java.net.URL;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
 import org.apache.nifi.annotation.configuration.DefaultSchedule;
 import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.components.ConfigurableComponent;
@@ -33,7 +41,7 @@ import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.controller.service.ControllerServiceProvider;
 import org.apache.nifi.controller.service.StandardConfigurationContext;
 import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.ComponentVariableRegistry;
 import org.apache.nifi.reporting.ReportingTask;
 import org.apache.nifi.scheduling.SchedulingStrategy;
 import org.apache.nifi.util.CharacterFilterUtils;
@@ -42,14 +50,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.core.annotation.AnnotationUtils;
 
-import java.net.URL;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-
 public abstract class AbstractReportingTaskNode extends AbstractConfiguredComponent implements ReportingTaskNode {
 
     private static final Logger LOG = LoggerFactory.getLogger(AbstractReportingTaskNode.class);
@@ -66,7 +66,7 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon
 
     public AbstractReportingTaskNode(final LoggableComponent<ReportingTask> reportingTask, final String id,
                                      final ControllerServiceProvider controllerServiceProvider, final ProcessScheduler processScheduler,
-                                     final ValidationContextFactory validationContextFactory, final VariableRegistry variableRegistry,
+                                     final ValidationContextFactory validationContextFactory, final ComponentVariableRegistry variableRegistry,
                                      final ReloadComponent reloadComponent) {
 
         this(reportingTask, id, controllerServiceProvider, processScheduler, validationContextFactory,
@@ -77,7 +77,7 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon
 
     public AbstractReportingTaskNode(final LoggableComponent<ReportingTask> reportingTask, final String id, final ControllerServiceProvider controllerServiceProvider,
                                      final ProcessScheduler processScheduler, final ValidationContextFactory validationContextFactory,
-                                     final String componentType, final String componentCanonicalClass, final VariableRegistry variableRegistry,
+                                     final String componentType, final String componentCanonicalClass, final ComponentVariableRegistry variableRegistry,
                                      final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
 
         super(id, validationContextFactory, controllerServiceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent, isExtensionMissing);

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java
index 53b7d15..40bdf8b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java
@@ -22,13 +22,13 @@ import org.apache.nifi.authorization.Resource;
 import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.resource.ResourceFactory;
 import org.apache.nifi.authorization.resource.ResourceType;
-import org.apache.nifi.controller.ReloadComponent;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.LoggableComponent;
 import org.apache.nifi.controller.ProcessScheduler;
+import org.apache.nifi.controller.ReloadComponent;
 import org.apache.nifi.controller.ReportingTaskNode;
 import org.apache.nifi.controller.ValidationContextFactory;
-import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.ComponentVariableRegistry;
 import org.apache.nifi.reporting.ReportingContext;
 import org.apache.nifi.reporting.ReportingTask;
 
@@ -38,14 +38,14 @@ public class StandardReportingTaskNode extends AbstractReportingTaskNode impleme
 
     public StandardReportingTaskNode(final LoggableComponent<ReportingTask> reportingTask, final String id, final FlowController controller,
                                      final ProcessScheduler processScheduler, final ValidationContextFactory validationContextFactory,
-                                     final VariableRegistry variableRegistry, final ReloadComponent reloadComponent) {
+                                     final ComponentVariableRegistry variableRegistry, final ReloadComponent reloadComponent) {
         super(reportingTask, id, controller, processScheduler, validationContextFactory, variableRegistry, reloadComponent);
         this.flowController = controller;
     }
 
     public StandardReportingTaskNode(final LoggableComponent<ReportingTask> reportingTask, final String id, final FlowController controller,
                                      final ProcessScheduler processScheduler, final ValidationContextFactory validationContextFactory,
-                                     final String componentType, final String canonicalClassName, final VariableRegistry variableRegistry,
+                                     final String componentType, final String canonicalClassName, final ComponentVariableRegistry variableRegistry,
                                      final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
         super(reportingTask, id, controller, processScheduler, validationContextFactory, componentType, canonicalClassName,
                 variableRegistry, reloadComponent, isExtensionMissing);

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java
index 0af8657..58d25bb 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java
@@ -45,7 +45,6 @@ import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.processor.SimpleProcessLogger;
 import org.apache.nifi.processor.StandardProcessContext;
 import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.registry.VariableRegistry;
 import org.apache.nifi.util.Connectables;
 import org.apache.nifi.util.FormatUtils;
 import org.apache.nifi.util.ReflectionUtils;
@@ -62,7 +61,6 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
     private final ProcessContextFactory contextFactory;
     private final AtomicInteger maxThreadCount;
     private final StringEncryptor encryptor;
-    private final VariableRegistry variableRegistry;
 
     private volatile String adminYieldDuration = "1 sec";
 
@@ -70,8 +68,7 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
     private final ConcurrentMap<Connectable, ScheduleState> scheduleStates = new ConcurrentHashMap<>();
 
     public EventDrivenSchedulingAgent(final FlowEngine flowEngine, final ControllerServiceProvider serviceProvider, final StateManagerProvider stateManagerProvider,
-                                      final EventDrivenWorkerQueue workerQueue, final ProcessContextFactory contextFactory, final int maxThreadCount, final StringEncryptor encryptor,
-                                      final VariableRegistry variableRegistry) {
+        final EventDrivenWorkerQueue workerQueue, final ProcessContextFactory contextFactory, final int maxThreadCount, final StringEncryptor encryptor) {
         super(flowEngine);
         this.serviceProvider = serviceProvider;
         this.stateManagerProvider = stateManagerProvider;
@@ -79,7 +76,6 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
         this.contextFactory = contextFactory;
         this.maxThreadCount = new AtomicInteger(maxThreadCount);
         this.encryptor = encryptor;
-        this.variableRegistry = variableRegistry;
 
         for (int i = 0; i < maxThreadCount; i++) {
             final Runnable eventDrivenTask = new EventDrivenTask(workerQueue);
@@ -188,8 +184,7 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
 
                 if (connectable instanceof ProcessorNode) {
                     final ProcessorNode procNode = (ProcessorNode) connectable;
-                    final StandardProcessContext standardProcessContext = new StandardProcessContext(procNode, serviceProvider,
-                        encryptor, getStateManager(connectable.getIdentifier()), variableRegistry);
+                    final StandardProcessContext standardProcessContext = new StandardProcessContext(procNode, serviceProvider, encryptor, getStateManager(connectable.getIdentifier()));
 
                     final long runNanos = procNode.getRunDuration(TimeUnit.NANOSECONDS);
                     final ProcessSessionFactory sessionFactory;

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java
index bd3c2bb..eb855d4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java
@@ -38,7 +38,6 @@ import org.apache.nifi.encrypt.StringEncryptor;
 import org.apache.nifi.engine.FlowEngine;
 import org.apache.nifi.processor.StandardProcessContext;
 import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.registry.VariableRegistry;
 import org.apache.nifi.util.FormatUtils;
 import org.quartz.CronExpression;
 import org.slf4j.Logger;
@@ -51,18 +50,15 @@ public class QuartzSchedulingAgent extends AbstractSchedulingAgent {
     private final FlowController flowController;
     private final ProcessContextFactory contextFactory;
     private final StringEncryptor encryptor;
-    private final VariableRegistry variableRegistry;
 
     private volatile String adminYieldDuration = "1 sec";
     private final Map<Object, List<AtomicBoolean>> canceledTriggers = new HashMap<>();
 
-    public QuartzSchedulingAgent(final FlowController flowController, final FlowEngine flowEngine, final ProcessContextFactory contextFactory, final StringEncryptor enryptor,
-                                 final VariableRegistry variableRegistry) {
+    public QuartzSchedulingAgent(final FlowController flowController, final FlowEngine flowEngine, final ProcessContextFactory contextFactory, final StringEncryptor enryptor) {
         super(flowEngine);
         this.flowController = flowController;
         this.contextFactory = contextFactory;
         this.encryptor = enryptor;
-        this.variableRegistry = variableRegistry;
     }
 
     private StateManager getStateManager(final String componentId) {
@@ -145,7 +141,7 @@ public class QuartzSchedulingAgent extends AbstractSchedulingAgent {
             if (connectable.getConnectableType() == ConnectableType.PROCESSOR) {
                 final ProcessorNode procNode = (ProcessorNode) connectable;
 
-                final StandardProcessContext standardProcContext = new StandardProcessContext(procNode, flowController, encryptor, getStateManager(connectable.getIdentifier()), variableRegistry);
+                final StandardProcessContext standardProcContext = new StandardProcessContext(procNode, flowController, encryptor, getStateManager(connectable.getIdentifier()));
                 ContinuallyRunProcessorTask runnableTask = new ContinuallyRunProcessorTask(this, procNode, flowController, contextFactory, scheduleState, standardProcContext);
                 continuallyRunTask = runnableTask;
             } else {

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
index 5368d37..122e50c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
@@ -53,7 +53,6 @@ import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.processor.Processor;
 import org.apache.nifi.processor.SimpleProcessLogger;
 import org.apache.nifi.processor.StandardProcessContext;
-import org.apache.nifi.registry.VariableRegistry;
 import org.apache.nifi.reporting.ReportingTask;
 import org.apache.nifi.scheduling.SchedulingStrategy;
 import org.apache.nifi.util.FormatUtils;
@@ -84,19 +83,16 @@ public final class StandardProcessScheduler implements ProcessScheduler {
     private final ScheduledExecutorService componentMonitoringThreadPool = new FlowEngine(8, "StandardProcessScheduler", true);
 
     private final StringEncryptor encryptor;
-    private final VariableRegistry variableRegistry;
 
     public StandardProcessScheduler(
             final ControllerServiceProvider controllerServiceProvider,
             final StringEncryptor encryptor,
             final StateManagerProvider stateManagerProvider,
-            final VariableRegistry variableRegistry,
             final NiFiProperties nifiProperties
     ) {
         this.controllerServiceProvider = controllerServiceProvider;
         this.encryptor = encryptor;
         this.stateManagerProvider = stateManagerProvider;
-        this.variableRegistry = variableRegistry;
 
         administrativeYieldDuration = nifiProperties.getAdministrativeYieldDuration();
         administrativeYieldMillis = FormatUtils.getTimeDuration(administrativeYieldDuration, TimeUnit.MILLISECONDS);
@@ -301,15 +297,17 @@ public final class StandardProcessScheduler implements ProcessScheduler {
      * @see StandardProcessorNode#start(ScheduledExecutorService, long, org.apache.nifi.processor.ProcessContext, Runnable).
      */
     @Override
-    public synchronized void startProcessor(final ProcessorNode procNode) {
+    public synchronized CompletableFuture<Void> startProcessor(final ProcessorNode procNode) {
         StandardProcessContext processContext = new StandardProcessContext(procNode, this.controllerServiceProvider,
-                this.encryptor, getStateManager(procNode.getIdentifier()), variableRegistry);
+            this.encryptor, getStateManager(procNode.getIdentifier()));
         final ScheduleState scheduleState = getScheduleState(requireNonNull(procNode));
 
+        final CompletableFuture<Void> future = new CompletableFuture<>();
         SchedulingAgentCallback callback = new SchedulingAgentCallback() {
             @Override
             public void trigger() {
                 getSchedulingAgent(procNode).schedule(procNode, scheduleState);
+                future.complete(null);
             }
 
             @Override
@@ -324,7 +322,9 @@ public final class StandardProcessScheduler implements ProcessScheduler {
             }
         };
 
+        LOG.info("Starting {}", procNode);
         procNode.start(this.componentLifeCycleThreadPool, this.administrativeYieldMillis, processContext, callback);
+        return future;
     }
 
     /**
@@ -335,12 +335,13 @@ public final class StandardProcessScheduler implements ProcessScheduler {
      * @see StandardProcessorNode#stop(ScheduledExecutorService, org.apache.nifi.processor.ProcessContext, SchedulingAgent, ScheduleState)
      */
     @Override
-    public synchronized void stopProcessor(final ProcessorNode procNode) {
+    public synchronized CompletableFuture<Void> stopProcessor(final ProcessorNode procNode) {
         StandardProcessContext processContext = new StandardProcessContext(procNode, this.controllerServiceProvider,
-                this.encryptor, getStateManager(procNode.getIdentifier()), variableRegistry);
+            this.encryptor, getStateManager(procNode.getIdentifier()));
         final ScheduleState state = getScheduleState(procNode);
 
-        procNode.stop(this.componentLifeCycleThreadPool, processContext, getSchedulingAgent(procNode), state);
+        LOG.info("Stopping {}", procNode);
+        return procNode.stop(this.componentLifeCycleThreadPool, processContext, getSchedulingAgent(procNode), state);
     }
 
     @Override
@@ -537,20 +538,35 @@ public final class StandardProcessScheduler implements ProcessScheduler {
 
     @Override
     public CompletableFuture<Void> enableControllerService(final ControllerServiceNode service) {
+        LOG.info("Enabling " + service);
         return service.enable(this.componentLifeCycleThreadPool, this.administrativeYieldMillis);
     }
 
     @Override
-    public void disableControllerService(final ControllerServiceNode service) {
-        service.disable(this.componentLifeCycleThreadPool);
+    public CompletableFuture<Void> disableControllerService(final ControllerServiceNode service) {
+        LOG.info("Disabling {}", service);
+        return service.disable(this.componentLifeCycleThreadPool);
     }
 
     @Override
-    public void disableControllerServices(final List<ControllerServiceNode> services) {
+    public CompletableFuture<Void> disableControllerServices(final List<ControllerServiceNode> services) {
+        if (services == null || services.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        CompletableFuture<Void> future = null;
         if (!requireNonNull(services).isEmpty()) {
             for (ControllerServiceNode controllerServiceNode : services) {
-                this.disableControllerService(controllerServiceNode);
+                final CompletableFuture<Void> serviceFuture = this.disableControllerService(controllerServiceNode);
+
+                if (future == null) {
+                    future = serviceFuture;
+                } else {
+                    future = CompletableFuture.allOf(future, serviceFuture);
+                }
             }
         }
+
+        return future;
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
index a82fde4..9dc329a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
@@ -37,7 +37,6 @@ import org.apache.nifi.engine.FlowEngine;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.StandardProcessContext;
 import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.registry.VariableRegistry;
 import org.apache.nifi.util.FormatUtils;
 import org.apache.nifi.util.NiFiProperties;
 import org.slf4j.Logger;
@@ -51,7 +50,6 @@ public class TimerDrivenSchedulingAgent extends AbstractSchedulingAgent {
     private final FlowController flowController;
     private final ProcessContextFactory contextFactory;
     private final StringEncryptor encryptor;
-    private final VariableRegistry variableRegistry;
 
     private volatile String adminYieldDuration = "1 sec";
 
@@ -60,13 +58,11 @@ public class TimerDrivenSchedulingAgent extends AbstractSchedulingAgent {
             final FlowEngine flowEngine,
             final ProcessContextFactory contextFactory,
             final StringEncryptor encryptor,
-            final VariableRegistry variableRegistry,
             final NiFiProperties nifiProperties) {
         super(flowEngine);
         this.flowController = flowController;
         this.contextFactory = contextFactory;
         this.encryptor = encryptor;
-        this.variableRegistry = variableRegistry;
 
         final String boredYieldDuration = nifiProperties.getBoredYieldDuration();
         try {
@@ -109,7 +105,7 @@ public class TimerDrivenSchedulingAgent extends AbstractSchedulingAgent {
             // Determine the task to run and create it.
             if (connectable.getConnectableType() == ConnectableType.PROCESSOR) {
                 final ProcessorNode procNode = (ProcessorNode) connectable;
-                final StandardProcessContext standardProcContext = new StandardProcessContext(procNode, flowController, encryptor, getStateManager(connectable.getIdentifier()), variableRegistry);
+                final StandardProcessContext standardProcContext = new StandardProcessContext(procNode, flowController, encryptor, getStateManager(connectable.getIdentifier()));
                 final ContinuallyRunProcessorTask runnableTask = new ContinuallyRunProcessorTask(this, procNode, flowController,
                         contextFactory, scheduleState, standardProcContext);
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowFromDOMFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowFromDOMFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowFromDOMFactory.java
index 61d9d29..f30a71e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowFromDOMFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowFromDOMFactory.java
@@ -144,6 +144,16 @@ public class FlowFromDOMFactory {
         dto.setPosition(getPosition(DomUtils.getChild(element, "position")));
         dto.setComments(getString(element, "comment"));
 
+        final Map<String, String> variables = new HashMap<>();
+        final NodeList variableList = DomUtils.getChildNodesByTagName(element, "variable");
+        for (int i = 0; i < variableList.getLength(); i++) {
+            final Element variableElement = (Element) variableList.item(i);
+            final String name = variableElement.getAttribute("name");
+            final String value = variableElement.getAttribute("value");
+            variables.put(name, value);
+        }
+        dto.setVariables(variables);
+
         final Set<ProcessorDTO> processors = new HashSet<>();
         final Set<ConnectionDTO> connections = new HashSet<>();
         final Set<FunnelDTO> funnels = new HashSet<>();

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/StandardFlowSerializer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/StandardFlowSerializer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/StandardFlowSerializer.java
index 4a9a0f7..2a8df96 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/StandardFlowSerializer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/StandardFlowSerializer.java
@@ -37,6 +37,8 @@ import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.persistence.TemplateSerializer;
 import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.registry.VariableDescriptor;
+import org.apache.nifi.registry.VariableRegistry;
 import org.apache.nifi.remote.RemoteGroupPort;
 import org.apache.nifi.remote.RootGroupPort;
 import org.apache.nifi.util.CharacterFilterUtils;
@@ -70,7 +72,7 @@ import java.util.concurrent.TimeUnit;
  */
 public class StandardFlowSerializer implements FlowSerializer {
 
-    private static final String MAX_ENCODING_VERSION = "1.1";
+    private static final String MAX_ENCODING_VERSION = "1.2";
 
     private final StringEncryptor encryptor;
 
@@ -202,6 +204,18 @@ public class StandardFlowSerializer implements FlowSerializer {
         for (final Template template : group.getTemplates()) {
             addTemplate(element, template);
         }
+
+        final VariableRegistry variableRegistry = group.getVariableRegistry();
+        for (final Map.Entry<VariableDescriptor, String> entry : variableRegistry.getVariableMap().entrySet()) {
+            addVariable(element, entry.getKey().getName(), entry.getValue());
+        }
+    }
+
+    private static void addVariable(final Element parentElement, final String variableName, final String variableValue) {
+        final Element variableElement = parentElement.getOwnerDocument().createElement("variable");
+        variableElement.setAttribute("name", variableName);
+        variableElement.setAttribute("value", variableValue);
+        parentElement.appendChild(variableElement);
     }
 
     private static void addBundle(final Element parentElement, final BundleCoordinate coordinate) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ServiceStateTransition.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ServiceStateTransition.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ServiceStateTransition.java
new file mode 100644
index 0000000..148b847
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ServiceStateTransition.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.controller.service;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+public class ServiceStateTransition {
+    private ControllerServiceState state = ControllerServiceState.DISABLED;
+    private final List<CompletableFuture<?>> enabledFutures = new ArrayList<>();
+    private final List<CompletableFuture<?>> disabledFutures = new ArrayList<>();
+
+
+    public synchronized boolean transitionToEnabling(final ControllerServiceState expectedState, final CompletableFuture<?> enabledFuture) {
+        if (expectedState != state) {
+            return false;
+        }
+
+        state = ControllerServiceState.ENABLING;
+        enabledFutures.add(enabledFuture);
+        return true;
+    }
+
+    public synchronized boolean enable() {
+        if (state != ControllerServiceState.ENABLING) {
+            return false;
+        }
+
+        state = ControllerServiceState.ENABLED;
+        enabledFutures.stream().forEach(future -> future.complete(null));
+        return true;
+    }
+
+    public synchronized boolean transitionToDisabling(final ControllerServiceState expectedState, final CompletableFuture<?> disabledFuture) {
+        if (expectedState != state) {
+            return false;
+        }
+
+        state = ControllerServiceState.DISABLING;
+        disabledFutures.add(disabledFuture);
+        return true;
+    }
+
+    public synchronized void disable() {
+        state = ControllerServiceState.DISABLED;
+        disabledFutures.stream().forEach(future -> future.complete(null));
+    }
+
+    public synchronized ControllerServiceState getState() {
+        return state;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
index fa4ab84..216a996 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
@@ -16,6 +16,24 @@
  */
 package org.apache.nifi.controller.service;
 
+import java.lang.reflect.InvocationTargetException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.annotation.behavior.Restricted;
 import org.apache.nifi.annotation.documentation.DeprecationNotice;
@@ -30,48 +48,30 @@ import org.apache.nifi.components.ConfigurableComponent;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.controller.AbstractConfiguredComponent;
-import org.apache.nifi.controller.ReloadComponent;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.ConfiguredComponent;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.LoggableComponent;
+import org.apache.nifi.controller.ReloadComponent;
 import org.apache.nifi.controller.ValidationContextFactory;
 import org.apache.nifi.controller.exception.ControllerServiceInstantiationException;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.processor.SimpleProcessLogger;
-import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.ComponentVariableRegistry;
 import org.apache.nifi.util.CharacterFilterUtils;
 import org.apache.nifi.util.ReflectionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.lang.reflect.InvocationTargetException;
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
 public class StandardControllerServiceNode extends AbstractConfiguredComponent implements ControllerServiceNode {
 
     private static final Logger LOG = LoggerFactory.getLogger(StandardControllerServiceNode.class);
 
     private final AtomicReference<ControllerServiceDetails> controllerServiceHolder = new AtomicReference<>(null);
     private final ControllerServiceProvider serviceProvider;
-    private final AtomicReference<ControllerServiceState> stateRef = new AtomicReference<>(ControllerServiceState.DISABLED);
+    private final ServiceStateTransition stateTransition = new ServiceStateTransition();
 
     private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
     private final Lock readLock = rwLock.readLock();
@@ -85,7 +85,7 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
 
     public StandardControllerServiceNode(final LoggableComponent<ControllerService> implementation, final LoggableComponent<ControllerService> proxiedControllerService,
                                          final ControllerServiceInvocationHandler invocationHandler, final String id, final ValidationContextFactory validationContextFactory,
-                                         final ControllerServiceProvider serviceProvider, final VariableRegistry variableRegistry, final ReloadComponent reloadComponent) {
+                                         final ControllerServiceProvider serviceProvider, final ComponentVariableRegistry variableRegistry, final ReloadComponent reloadComponent) {
 
         this(implementation, proxiedControllerService, invocationHandler, id, validationContextFactory, serviceProvider,
             implementation.getComponent().getClass().getSimpleName(), implementation.getComponent().getClass().getCanonicalName(), variableRegistry, reloadComponent, false);
@@ -94,7 +94,7 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
     public StandardControllerServiceNode(final LoggableComponent<ControllerService> implementation, final LoggableComponent<ControllerService> proxiedControllerService,
                                          final ControllerServiceInvocationHandler invocationHandler, final String id, final ValidationContextFactory validationContextFactory,
                                          final ControllerServiceProvider serviceProvider, final String componentType, final String componentCanonicalClass,
-                                         final VariableRegistry variableRegistry, final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
+                                         final ComponentVariableRegistry variableRegistry, final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
 
         super(id, validationContextFactory, serviceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent, isExtensionMissing);
         this.serviceProvider = serviceProvider;
@@ -363,7 +363,7 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
 
     @Override
     public ControllerServiceState getState() {
-        return stateRef.get();
+        return stateTransition.getState();
     }
 
     @Override
@@ -394,7 +394,7 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
     public CompletableFuture<Void> enable(final ScheduledExecutorService scheduler, final long administrativeYieldMillis) {
         final CompletableFuture<Void> future = new CompletableFuture<>();
 
-        if (this.stateRef.compareAndSet(ControllerServiceState.DISABLED, ControllerServiceState.ENABLING)) {
+        if (this.stateTransition.transitionToEnabling(ControllerServiceState.DISABLED, future)) {
             synchronized (active) {
                 this.active.set(true);
             }
@@ -410,17 +410,15 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
 
                         boolean shouldEnable = false;
                         synchronized (active) {
-                            shouldEnable = active.get() && stateRef.compareAndSet(ControllerServiceState.ENABLING, ControllerServiceState.ENABLED);
+                            shouldEnable = active.get() && stateTransition.enable();
                         }
 
-                        future.complete(null);
-
                         if (!shouldEnable) {
                             LOG.debug("Disabling service " + this + " after it has been enabled due to disable action being initiated.");
                             // Can only happen if user initiated DISABLE operation before service finished enabling. It's state will be
                             // set to DISABLING (see disable() operation)
                             invokeDisable(configContext);
-                            stateRef.set(ControllerServiceState.DISABLED);
+                            stateTransition.disable();
                         }
                     } catch (Exception e) {
                         future.completeExceptionally(e);
@@ -437,7 +435,7 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
                             try (final NarCloseable nc = NarCloseable.withComponentNarLoader(getControllerServiceImplementation().getClass(), getIdentifier())) {
                                 ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnDisabled.class, getControllerServiceImplementation(), configContext);
                             }
-                            stateRef.set(ControllerServiceState.DISABLED);
+                            stateTransition.disable();
                         }
                     }
                 }
@@ -464,7 +462,7 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
      * DISABLED state.
      */
     @Override
-    public void disable(ScheduledExecutorService scheduler) {
+    public CompletableFuture<Void> disable(ScheduledExecutorService scheduler) {
         /*
          * The reason for synchronization is to ensure consistency of the
          * service state when another thread is in the middle of enabling this
@@ -475,7 +473,8 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
             this.active.set(false);
         }
 
-        if (this.stateRef.compareAndSet(ControllerServiceState.ENABLED, ControllerServiceState.DISABLING)) {
+        final CompletableFuture<Void> future = new CompletableFuture<>();
+        if (this.stateTransition.transitionToDisabling(ControllerServiceState.ENABLED, future)) {
             final ConfigurationContext configContext = new StandardConfigurationContext(this, this.serviceProvider, null, getVariableRegistry());
             scheduler.execute(new Runnable() {
                 @Override
@@ -483,13 +482,15 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
                     try {
                         invokeDisable(configContext);
                     } finally {
-                        stateRef.set(ControllerServiceState.DISABLED);
+                        stateTransition.disable();
                     }
                 }
             });
         } else {
-            this.stateRef.compareAndSet(ControllerServiceState.ENABLING, ControllerServiceState.DISABLING);
+            this.stateTransition.transitionToDisabling(ControllerServiceState.ENABLING, future);
         }
+
+        return future;
     }
 
     /**
@@ -515,7 +516,7 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
     @Override
     public Collection<ValidationResult> getValidationErrors(Set<String> serviceIdentifiersNotToValidate) {
         Collection<ValidationResult> results = null;
-        if (stateRef.get() == ControllerServiceState.DISABLED) {
+        if (getState() == ControllerServiceState.DISABLED) {
             results = super.getValidationErrors(serviceIdentifiersNotToValidate);
         }
         return results != null ? results : Collections.emptySet();

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
index 5c4e394..0745ed0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
@@ -30,6 +30,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.Future;
@@ -62,7 +63,9 @@ import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.processor.SimpleProcessLogger;
 import org.apache.nifi.processor.StandardValidationContextFactory;
+import org.apache.nifi.registry.ComponentVariableRegistry;
 import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.variable.StandardComponentVariableRegistry;
 import org.apache.nifi.reporting.BulletinRepository;
 import org.apache.nifi.reporting.Severity;
 import org.apache.nifi.util.NiFiProperties;
@@ -148,8 +151,9 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
             final LoggableComponent<ControllerService> originalLoggableComponent = new LoggableComponent<>(originalService, bundleCoordinate, serviceLogger);
             final LoggableComponent<ControllerService> proxiedLoggableComponent = new LoggableComponent<>(proxiedService, bundleCoordinate, serviceLogger);
 
+            final ComponentVariableRegistry componentVarRegistry = new StandardComponentVariableRegistry(this.variableRegistry);
             final ControllerServiceNode serviceNode = new StandardControllerServiceNode(originalLoggableComponent, proxiedLoggableComponent, invocationHandler,
-                    id, validationContextFactory, this, variableRegistry, flowController);
+                    id, validationContextFactory, this, componentVarRegistry, flowController);
             serviceNode.setName(rawClass.getSimpleName());
 
             invocationHandler.setServiceNode(serviceNode);
@@ -226,8 +230,9 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
 
         final LoggableComponent<ControllerService> proxiedLoggableComponent = new LoggableComponent<>(proxiedService, bundleCoordinate, null);
 
+        final ComponentVariableRegistry componentVarRegistry = new StandardComponentVariableRegistry(this.variableRegistry);
         final ControllerServiceNode serviceNode = new StandardControllerServiceNode(proxiedLoggableComponent, proxiedLoggableComponent, invocationHandler, id,
-                new StandardValidationContextFactory(this, variableRegistry), this, componentType, type, variableRegistry, flowController, true);
+                new StandardValidationContextFactory(this, variableRegistry), this, componentType, type, componentVarRegistry, flowController, true);
 
         serviceCache.putIfAbsent(id, serviceNode);
         return serviceNode;
@@ -235,9 +240,8 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
 
     @Override
     public Set<ConfiguredComponent> disableReferencingServices(final ControllerServiceNode serviceNode) {
-        // Get a list of all Controller Services that need to be disabled, in the order that they need to be
-        // disabled.
-        final List<ControllerServiceNode> toDisable = findRecursiveReferences(serviceNode, ControllerServiceNode.class);
+        // Get a list of all Controller Services that need to be disabled, in the order that they need to be disabled.
+        final List<ControllerServiceNode> toDisable = serviceNode.getReferences().findRecursiveReferences(ControllerServiceNode.class);
 
         final Set<ControllerServiceNode> serviceSet = new HashSet<>(toDisable);
 
@@ -258,8 +262,8 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
     public Set<ConfiguredComponent> scheduleReferencingComponents(final ControllerServiceNode serviceNode) {
         // find all of the schedulable components (processors, reporting tasks) that refer to this Controller Service,
         // or a service that references this controller service, etc.
-        final List<ProcessorNode> processors = findRecursiveReferences(serviceNode, ProcessorNode.class);
-        final List<ReportingTaskNode> reportingTasks = findRecursiveReferences(serviceNode, ReportingTaskNode.class);
+        final List<ProcessorNode> processors = serviceNode.getReferences().findRecursiveReferences(ProcessorNode.class);
+        final List<ReportingTaskNode> reportingTasks = serviceNode.getReferences().findRecursiveReferences(ReportingTaskNode.class);
 
         final Set<ConfiguredComponent> updated = new HashSet<>();
 
@@ -298,8 +302,8 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
     public Set<ConfiguredComponent> unscheduleReferencingComponents(final ControllerServiceNode serviceNode) {
         // find all of the schedulable components (processors, reporting tasks) that refer to this Controller Service,
         // or a service that references this controller service, etc.
-        final List<ProcessorNode> processors = findRecursiveReferences(serviceNode, ProcessorNode.class);
-        final List<ReportingTaskNode> reportingTasks = findRecursiveReferences(serviceNode, ReportingTaskNode.class);
+        final List<ProcessorNode> processors = serviceNode.getReferences().findRecursiveReferences(ProcessorNode.class);
+        final List<ReportingTaskNode> reportingTasks = serviceNode.getReferences().findRecursiveReferences(ReportingTaskNode.class);
 
         final Set<ConfiguredComponent> updated = new HashSet<>();
 
@@ -333,7 +337,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
     }
 
     @Override
-    public Future<Void> enableControllerService(final ControllerServiceNode serviceNode) {
+    public CompletableFuture<Void> enableControllerService(final ControllerServiceNode serviceNode) {
         serviceNode.verifyCanEnable();
         return processScheduler.enableControllerService(serviceNode);
     }
@@ -450,9 +454,9 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
     }
 
     @Override
-    public void disableControllerService(final ControllerServiceNode serviceNode) {
+    public CompletableFuture<Void> disableControllerService(final ControllerServiceNode serviceNode) {
         serviceNode.verifyCanDisable();
-        processScheduler.disableControllerService(serviceNode);
+        return processScheduler.disableControllerService(serviceNode);
     }
 
     @Override
@@ -589,43 +593,10 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
         return allServices;
     }
 
-    /**
-     * Returns a List of all components that reference the given referencedNode
-     * (either directly or indirectly through another service) that are also of
-     * the given componentType. The list that is returned is in the order in
-     * which they will need to be 'activated' (enabled/started).
-     *
-     * @param referencedNode node
-     * @param componentType type
-     * @return list of components
-     */
-    private <T> List<T> findRecursiveReferences(final ControllerServiceNode referencedNode, final Class<T> componentType) {
-        final List<T> references = new ArrayList<>();
-
-        for (final ConfiguredComponent referencingComponent : referencedNode.getReferences().getReferencingComponents()) {
-            if (componentType.isAssignableFrom(referencingComponent.getClass())) {
-                references.add(componentType.cast(referencingComponent));
-            }
-
-            if (referencingComponent instanceof ControllerServiceNode) {
-                final ControllerServiceNode referencingNode = (ControllerServiceNode) referencingComponent;
-
-                // find components recursively that depend on referencingNode.
-                final List<T> recursive = findRecursiveReferences(referencingNode, componentType);
-
-                // For anything that depends on referencing node, we want to add it to the list, but we know
-                // that it must come after the referencing node, so we first remove any existing occurrence.
-                references.removeAll(recursive);
-                references.addAll(recursive);
-            }
-        }
-
-        return references;
-    }
 
     @Override
     public Set<ConfiguredComponent> enableReferencingServices(final ControllerServiceNode serviceNode) {
-        final List<ControllerServiceNode> recursiveReferences = findRecursiveReferences(serviceNode, ControllerServiceNode.class);
+        final List<ControllerServiceNode> recursiveReferences = serviceNode.getReferences().findRecursiveReferences(ControllerServiceNode.class);
         logger.debug("Enabling the following Referencing Services for {}: {}", serviceNode, recursiveReferences);
         return enableReferencingServices(serviceNode, recursiveReferences);
     }
@@ -658,7 +629,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
 
     @Override
     public void verifyCanEnableReferencingServices(final ControllerServiceNode serviceNode) {
-        final List<ControllerServiceNode> referencingServices = findRecursiveReferences(serviceNode, ControllerServiceNode.class);
+        final List<ControllerServiceNode> referencingServices = serviceNode.getReferences().findRecursiveReferences(ControllerServiceNode.class);
         final Set<ControllerServiceNode> referencingServiceSet = new HashSet<>(referencingServices);
 
         for (final ControllerServiceNode referencingService : referencingServices) {
@@ -668,9 +639,9 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
 
     @Override
     public void verifyCanScheduleReferencingComponents(final ControllerServiceNode serviceNode) {
-        final List<ControllerServiceNode> referencingServices = findRecursiveReferences(serviceNode, ControllerServiceNode.class);
-        final List<ReportingTaskNode> referencingReportingTasks = findRecursiveReferences(serviceNode, ReportingTaskNode.class);
-        final List<ProcessorNode> referencingProcessors = findRecursiveReferences(serviceNode, ProcessorNode.class);
+        final List<ControllerServiceNode> referencingServices = serviceNode.getReferences().findRecursiveReferences(ControllerServiceNode.class);
+        final List<ReportingTaskNode> referencingReportingTasks = serviceNode.getReferences().findRecursiveReferences(ReportingTaskNode.class);
+        final List<ProcessorNode> referencingProcessors = serviceNode.getReferences().findRecursiveReferences(ProcessorNode.class);
 
         final Set<ControllerServiceNode> referencingServiceSet = new HashSet<>(referencingServices);
 
@@ -689,9 +660,8 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
 
     @Override
     public void verifyCanDisableReferencingServices(final ControllerServiceNode serviceNode) {
-        // Get a list of all Controller Services that need to be disabled, in the order that they need to be
-        // disabled.
-        final List<ControllerServiceNode> toDisable = findRecursiveReferences(serviceNode, ControllerServiceNode.class);
+        // Get a list of all Controller Services that need to be disabled, in the order that they need to be disabled.
+        final List<ControllerServiceNode> toDisable = serviceNode.getReferences().findRecursiveReferences(ControllerServiceNode.class);
         final Set<ControllerServiceNode> serviceSet = new HashSet<>(toDisable);
 
         for (final ControllerServiceNode nodeToDisable : toDisable) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceReference.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceReference.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceReference.java
index d2f3833..285b8dc 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceReference.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceReference.java
@@ -16,8 +16,10 @@
  */
 package org.apache.nifi.controller.service;
 
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Set;
 
 import org.apache.nifi.controller.ConfiguredComponent;
@@ -101,4 +103,35 @@ public class StandardControllerServiceReference implements ControllerServiceRefe
 
         return references;
     }
+
+
+    @Override
+    public <T> List<T> findRecursiveReferences(final Class<T> componentType) {
+        return findRecursiveReferences(referenced, componentType);
+    }
+
+    private <T> List<T> findRecursiveReferences(final ControllerServiceNode referencedNode, final Class<T> componentType) {
+        final List<T> references = new ArrayList<>();
+
+        for (final ConfiguredComponent referencingComponent : referencedNode.getReferences().getReferencingComponents()) {
+            if (componentType.isAssignableFrom(referencingComponent.getClass())) {
+                references.add(componentType.cast(referencingComponent));
+            }
+
+            if (referencingComponent instanceof ControllerServiceNode) {
+                final ControllerServiceNode referencingNode = (ControllerServiceNode) referencingComponent;
+
+                // find components recursively that depend on referencingNode.
+                final List<T> recursive = findRecursiveReferences(referencingNode, componentType);
+
+                // For anything that depends on referencing node, we want to add it to the list, but we know
+                // that it must come after the referencing node, so we first remove any existing occurrence.
+                references.removeAll(recursive);
+                references.addAll(recursive);
+            }
+        }
+
+        return references;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
index 1ef3e8b..3957955 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
@@ -16,6 +16,24 @@
  */
 package org.apache.nifi.fingerprint;
 
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import javax.xml.XMLConstants;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.validation.Schema;
+import javax.xml.validation.SchemaFactory;
+
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.components.ConfigurableComponent;
@@ -38,23 +56,6 @@ import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 import org.xml.sax.SAXException;
 
-import javax.xml.XMLConstants;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.security.NoSuchAlgorithmException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.Map;
-import java.util.SortedMap;
-import java.util.TreeMap;
-
 /**
  * <p>Creates a fingerprint of a flow.xml. The order of elements or attributes in the flow.xml does not influence the fingerprint generation.
  *
@@ -324,9 +325,22 @@ public class FingerprintFactory {
             addFunnelFingerprint(builder, funnelElem);
         }
 
+        // add variables
+        final NodeList variableElems = DomUtils.getChildNodesByTagName(processGroupElem, "variable");
+        final List<Element> sortedVarList = sortElements(variableElems, getVariableNameComparator());
+        for (final Element varElem : sortedVarList) {
+            addVariableFingerprint(builder, varElem);
+        }
+
         return builder;
     }
 
+    private void addVariableFingerprint(final StringBuilder builder, final Element variableElement) {
+        final String variableName = variableElement.getAttribute("name");
+        final String variableValue = variableElement.getAttribute("value");
+        builder.append(variableName).append("=").append(variableValue);
+    }
+
     private StringBuilder addFlowFileProcessorFingerprint(final StringBuilder builder, final Element processorElem) throws FingerprintException {
         // id
         appendFirstValue(builder, DomUtils.getChildNodesByTagName(processorElem, "id"));
@@ -662,6 +676,27 @@ public class FingerprintFactory {
         };
     }
 
+    private Comparator<Element> getVariableNameComparator() {
+        return new Comparator<Element>() {
+            @Override
+            public int compare(final Element e1, final Element e2) {
+                if (e1 == null && e2 == null) {
+                    return 0;
+                }
+                if (e1 == null) {
+                    return 1;
+                }
+                if (e2 == null) {
+                    return -1;
+                }
+
+                final String varName1 = e1.getAttribute("name");
+                final String varName2 = e2.getAttribute("name");
+                return varName1.compareTo(varName2);
+            }
+        };
+    }
+
     private Comparator<Element> getProcessorPropertiesComparator() {
         return new Comparator<Element>() {
             @Override

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
index 2907704..2b7b51d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
@@ -16,13 +16,31 @@
  */
 package org.apache.nifi.groups;
 
-import com.google.common.collect.Sets;
+import static java.util.Objects.requireNonNull;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.builder.HashCodeBuilder;
 import org.apache.commons.lang3.builder.ToStringBuilder;
 import org.apache.commons.lang3.builder.ToStringStyle;
 import org.apache.nifi.annotation.lifecycle.OnRemoved;
 import org.apache.nifi.annotation.lifecycle.OnShutdown;
+import org.apache.nifi.attribute.expression.language.Query;
+import org.apache.nifi.attribute.expression.language.VariableImpact;
 import org.apache.nifi.authorization.Resource;
 import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.resource.ResourceFactory;
@@ -39,6 +57,7 @@ import org.apache.nifi.connectable.Port;
 import org.apache.nifi.connectable.Position;
 import org.apache.nifi.connectable.Positionable;
 import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.controller.ConfiguredComponent;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
@@ -50,13 +69,15 @@ import org.apache.nifi.controller.label.Label;
 import org.apache.nifi.controller.scheduling.StandardProcessScheduler;
 import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.controller.service.ControllerServiceProvider;
+import org.apache.nifi.controller.service.ControllerServiceReference;
 import org.apache.nifi.controller.service.StandardConfigurationContext;
 import org.apache.nifi.encrypt.StringEncryptor;
 import org.apache.nifi.logging.LogRepositoryFactory;
 import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.processor.StandardProcessContext;
-import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.VariableDescriptor;
+import org.apache.nifi.registry.variable.MutableVariableRegistry;
 import org.apache.nifi.remote.RemoteGroupPort;
 import org.apache.nifi.remote.RootGroupPort;
 import org.apache.nifi.util.NiFiProperties;
@@ -66,20 +87,7 @@ import org.apache.nifi.web.api.dto.TemplateDTO;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import static java.util.Objects.requireNonNull;
+import com.google.common.collect.Sets;
 
 public final class StandardProcessGroup implements ProcessGroup {
 
@@ -104,7 +112,7 @@ public final class StandardProcessGroup implements ProcessGroup {
     private final Map<String, ControllerServiceNode> controllerServices = new HashMap<>();
     private final Map<String, Template> templates = new HashMap<>();
     private final StringEncryptor encryptor;
-    private final VariableRegistry variableRegistry;
+    private final MutableVariableRegistry variableRegistry;
 
     private final ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock();
     private final Lock readLock = rwLock.readLock();
@@ -114,7 +122,7 @@ public final class StandardProcessGroup implements ProcessGroup {
 
     public StandardProcessGroup(final String id, final ControllerServiceProvider serviceProvider, final StandardProcessScheduler scheduler,
             final NiFiProperties nifiProps, final StringEncryptor encryptor, final FlowController flowController,
-            final VariableRegistry variableRegistry) {
+            final MutableVariableRegistry variableRegistry) {
         this.id = id;
         this.controllerServiceProvider = serviceProvider;
         this.parent = new AtomicReference<>();
@@ -361,7 +369,7 @@ public final class StandardProcessGroup implements ProcessGroup {
     private void shutdown(final ProcessGroup procGroup) {
         for (final ProcessorNode node : procGroup.getProcessors()) {
             try (final NarCloseable x = NarCloseable.withComponentNarLoader(node.getProcessor().getClass(), node.getIdentifier())) {
-                final StandardProcessContext processContext = new StandardProcessContext(node, controllerServiceProvider, encryptor, getStateManager(node.getIdentifier()), variableRegistry);
+                final StandardProcessContext processContext = new StandardProcessContext(node, controllerServiceProvider, encryptor, getStateManager(node.getIdentifier()));
                 ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, node.getProcessor(), processContext);
             }
         }
@@ -548,6 +556,8 @@ public final class StandardProcessGroup implements ProcessGroup {
         writeLock.lock();
         try {
             group.setParent(this);
+            group.getVariableRegistry().setParent(getVariableRegistry());
+
             processGroups.put(Objects.requireNonNull(group).getIdentifier(), group);
             flowController.onProcessGroupAdded(group);
         } finally {
@@ -709,6 +719,7 @@ public final class StandardProcessGroup implements ProcessGroup {
             }
 
             processor.setProcessGroup(this);
+            processor.getVariableRegistry().setParent(getVariableRegistry());
             processors.put(processorId, processor);
             flowController.onProcessorAdded(processor);
         } finally {
@@ -732,7 +743,7 @@ public final class StandardProcessGroup implements ProcessGroup {
             }
 
             try (final NarCloseable x = NarCloseable.withComponentNarLoader(processor.getProcessor().getClass(), processor.getIdentifier())) {
-                final StandardProcessContext processContext = new StandardProcessContext(processor, controllerServiceProvider, encryptor, getStateManager(processor.getIdentifier()), variableRegistry);
+                final StandardProcessContext processContext = new StandardProcessContext(processor, controllerServiceProvider, encryptor, getStateManager(processor.getIdentifier()));
                 ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, processor.getProcessor(), processContext);
             } catch (final Exception e) {
                 throw new ComponentLifeCycleException("Failed to invoke 'OnRemoved' methods of processor with id " + processor.getIdentifier(), e);
@@ -1081,7 +1092,7 @@ public final class StandardProcessGroup implements ProcessGroup {
     }
 
     @Override
-    public void startProcessor(final ProcessorNode processor) {
+    public CompletableFuture<Void> startProcessor(final ProcessorNode processor) {
         readLock.lock();
         try {
             if (getProcessor(processor.getIdentifier()) == null) {
@@ -1092,10 +1103,10 @@ public final class StandardProcessGroup implements ProcessGroup {
             if (state == ScheduledState.DISABLED) {
                 throw new IllegalStateException("Processor is disabled");
             } else if (state == ScheduledState.RUNNING) {
-                return;
+                return CompletableFuture.completedFuture(null);
             }
 
-            scheduler.startProcessor(processor);
+            return scheduler.startProcessor(processor);
         } finally {
             readLock.unlock();
         }
@@ -1162,7 +1173,7 @@ public final class StandardProcessGroup implements ProcessGroup {
     }
 
     @Override
-    public void stopProcessor(final ProcessorNode processor) {
+    public CompletableFuture<Void> stopProcessor(final ProcessorNode processor) {
         readLock.lock();
         try {
             if (!processors.containsKey(processor.getIdentifier())) {
@@ -1173,10 +1184,10 @@ public final class StandardProcessGroup implements ProcessGroup {
             if (state == ScheduledState.DISABLED) {
                 throw new IllegalStateException("Processor is disabled");
             } else if (state == ScheduledState.STOPPED) {
-                return;
+                return CompletableFuture.completedFuture(null);
             }
 
-            scheduler.stopProcessor(processor);
+            return scheduler.stopProcessor(processor);
         } finally {
             readLock.unlock();
         }
@@ -1854,6 +1865,7 @@ public final class StandardProcessGroup implements ProcessGroup {
             }
 
             service.setProcessGroup(this);
+            service.getVariableRegistry().setParent(getVariableRegistry());
             this.controllerServices.put(service.getIdentifier(), service);
             LOG.info("{} added to {}", service, this);
         } finally {
@@ -2583,4 +2595,129 @@ public final class StandardProcessGroup implements ProcessGroup {
             readLock.unlock();
         }
     }
+
+    @Override
+    public MutableVariableRegistry getVariableRegistry() {
+        return variableRegistry;
+    }
+
+    @Override
+    public void verifyCanUpdateVariables(final Map<String, String> updatedVariables) {
+        if (updatedVariables == null || updatedVariables.isEmpty()) {
+            return;
+        }
+
+        readLock.lock();
+        try {
+            final Set<String> updatedVariableNames = getUpdatedVariables(updatedVariables);
+            if (updatedVariableNames.isEmpty()) {
+                return;
+            }
+
+            for (final ProcessorNode processor : findAllProcessors()) {
+                if (!processor.isRunning()) {
+                    continue;
+                }
+
+                for (final String variableName : updatedVariableNames) {
+                    for (final VariableImpact impact : getVariableImpact(processor)) {
+                        if (impact.isImpacted(variableName)) {
+                            throw new IllegalStateException("Cannot update variable '" + variableName + "' because it is referenced by " + processor + ", which is currently running");
+                        }
+                    }
+                }
+            }
+
+            for (final ControllerServiceNode service : findAllControllerServices()) {
+                if (!service.isActive()) {
+                    continue;
+                }
+
+                for (final String variableName : updatedVariableNames) {
+                    for (final VariableImpact impact : getVariableImpact(service)) {
+                        if (impact.isImpacted(variableName)) {
+                            throw new IllegalStateException("Cannot update variable '" + variableName + "' because it is referenced by " + service + ", which is currently running");
+                        }
+                    }
+                }
+            }
+        } finally {
+            readLock.unlock();
+        }
+    }
+
+    @Override
+    public Set<ConfiguredComponent> getComponentsAffectedByVariable(final String variableName) {
+        final Set<ConfiguredComponent> affected = new HashSet<>();
+
+        // Determine any Processors that references the variable
+        for (final ProcessorNode processor : findAllProcessors()) {
+            for (final VariableImpact impact : getVariableImpact(processor)) {
+                if (impact.isImpacted(variableName)) {
+                    affected.add(processor);
+                }
+            }
+        }
+
+        // Determine any Controller Service that references the variable. If Service A references a variable,
+        // then that means that any other component that references that service is also affected, so recursively
+        // find any references to that service and add it.
+        for (final ControllerServiceNode service : findAllControllerServices()) {
+            for (final VariableImpact impact : getVariableImpact(service)) {
+                if (impact.isImpacted(variableName)) {
+                    affected.add(service);
+
+                    final ControllerServiceReference reference = service.getReferences();
+                    affected.addAll(reference.findRecursiveReferences(ConfiguredComponent.class));
+                }
+            }
+        }
+
+        return affected;
+    }
+
+
+    private Set<String> getUpdatedVariables(final Map<String, String> newVariableValues) {
+        final Set<String> updatedVariableNames = new HashSet<>();
+
+        final MutableVariableRegistry registry = getVariableRegistry();
+        for (final Map.Entry<String, String> entry : newVariableValues.entrySet()) {
+            final String varName = entry.getKey();
+            final String newValue = entry.getValue();
+
+            final String curValue = registry.getVariableValue(varName);
+            if (!Objects.equals(newValue, curValue)) {
+                updatedVariableNames.add(varName);
+            }
+        }
+
+        return updatedVariableNames;
+    }
+
+    private List<VariableImpact> getVariableImpact(final ConfiguredComponent component) {
+        return component.getProperties().values().stream()
+            .map(propVal -> Query.prepare(propVal).getVariableImpact())
+            .collect(Collectors.toList());
+    }
+
+    @Override
+    public void setVariables(final Map<String, String> variables) {
+        writeLock.lock();
+        try {
+            verifyCanUpdateVariables(variables);
+
+            if (variables == null) {
+                return;
+            }
+
+            final Map<VariableDescriptor, String> variableMap = new HashMap<>();
+            variables.entrySet().stream() // cannot use Collectors.toMap because value may be null
+                .forEach(entry -> variableMap.put(new VariableDescriptor(entry.getKey()), entry.getValue()));
+
+            variableRegistry.setVariables(variableMap);
+        } finally {
+            writeLock.unlock();
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
index 2714392..dfba330 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
@@ -26,8 +26,8 @@ import java.util.Set;
 
 import org.apache.nifi.attribute.expression.language.PreparedQuery;
 import org.apache.nifi.attribute.expression.language.Query;
-import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
 import org.apache.nifi.attribute.expression.language.Query.Range;
+import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.components.state.StateManager;
@@ -37,7 +37,6 @@ import org.apache.nifi.controller.ControllerServiceLookup;
 import org.apache.nifi.controller.ProcessorNode;
 import org.apache.nifi.controller.service.ControllerServiceProvider;
 import org.apache.nifi.encrypt.StringEncryptor;
-import org.apache.nifi.registry.VariableRegistry;
 import org.apache.nifi.util.Connectables;
 
 public class StandardProcessContext implements ProcessContext, ControllerServiceLookup {
@@ -47,15 +46,12 @@ public class StandardProcessContext implements ProcessContext, ControllerService
     private final Map<PropertyDescriptor, PreparedQuery> preparedQueries;
     private final StringEncryptor encryptor;
     private final StateManager stateManager;
-    private final VariableRegistry variableRegistry;
 
-    public StandardProcessContext(final ProcessorNode processorNode, final ControllerServiceProvider controllerServiceProvider, final StringEncryptor encryptor, final StateManager stateManager,
-                                  final VariableRegistry variableRegistry) {
+    public StandardProcessContext(final ProcessorNode processorNode, final ControllerServiceProvider controllerServiceProvider, final StringEncryptor encryptor, final StateManager stateManager) {
         this.procNode = processorNode;
         this.controllerServiceProvider = controllerServiceProvider;
         this.encryptor = encryptor;
         this.stateManager = stateManager;
-        this.variableRegistry = variableRegistry;
 
         preparedQueries = new HashMap<>();
         for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getProperties().entrySet()) {
@@ -93,12 +89,12 @@ public class StandardProcessContext implements ProcessContext, ControllerService
         final String setPropertyValue = procNode.getProperty(descriptor);
         final String propValue = (setPropertyValue == null) ? descriptor.getDefaultValue() : setPropertyValue;
 
-        return new StandardPropertyValue(propValue, this, preparedQueries.get(descriptor), variableRegistry);
+        return new StandardPropertyValue(propValue, this, preparedQueries.get(descriptor), procNode.getVariableRegistry());
     }
 
     @Override
     public PropertyValue newPropertyValue(final String rawValue) {
-        return new StandardPropertyValue(rawValue, this, Query.prepare(rawValue), variableRegistry);
+        return new StandardPropertyValue(rawValue, this, Query.prepare(rawValue), procNode.getVariableRegistry());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java
index 662169c..2f38aee 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java
@@ -53,8 +53,8 @@ public class StandardValidationContext implements ValidationContext {
     private final String componentId;
 
     public StandardValidationContext(final ControllerServiceProvider controllerServiceProvider, final Map<PropertyDescriptor, String> properties,
-            final String annotationData, final String groupId, final String componentId, VariableRegistry variableRegistry) {
-        this(controllerServiceProvider, Collections.<String> emptySet(), properties, annotationData, groupId, componentId,variableRegistry);
+            final String annotationData, final String groupId, final String componentId, final VariableRegistry variableRegistry) {
+        this(controllerServiceProvider, Collections.<String> emptySet(), properties, annotationData, groupId, componentId, variableRegistry);
     }
 
     public StandardValidationContext(
@@ -63,7 +63,8 @@ public class StandardValidationContext implements ValidationContext {
             final Map<PropertyDescriptor, String> properties,
             final String annotationData,
             final String groupId,
-            final String componentId, VariableRegistry variableRegistry) {
+            final String componentId,
+            final VariableRegistry variableRegistry) {
         this.controllerServiceProvider = controllerServiceProvider;
         this.properties = new HashMap<>(properties);
         this.annotationData = annotationData;


[4/9] nifi git commit: NIFI-4224: - Initial implementation of Process Group level Variable Registry - Updated to incorporate PR Feedback - Changed log message because slf4j-simple apparently has a memory leak; passing a String instead of passing in the C

Posted by mc...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/variable/FileBasedVariableRegistry.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/variable/FileBasedVariableRegistry.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/variable/FileBasedVariableRegistry.java
new file mode 100644
index 0000000..fb7a19d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/variable/FileBasedVariableRegistry.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.variable;
+
+import java.io.BufferedInputStream;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.nifi.registry.VariableDescriptor;
+import org.apache.nifi.registry.VariableRegistry;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A file based variable registry that loads all properties from files specified
+ * during construction and is backed by system properties and environment
+ * variables accessible to the JVM.
+ */
+public class FileBasedVariableRegistry implements VariableRegistry {
+
+    private final static Logger LOG = LoggerFactory.getLogger(FileBasedVariableRegistry.class);
+    final Map<VariableDescriptor, String> map;
+
+    public FileBasedVariableRegistry(final Path[] propertiesPaths) {
+        final Map<VariableDescriptor, String> newMap = new HashMap<>(VariableRegistry.ENVIRONMENT_SYSTEM_REGISTRY.getVariableMap());
+        final int systemEnvPropCount = newMap.size();
+        int totalPropertiesLoaded = systemEnvPropCount;
+        LOG.info("Loaded {} properties from system properties and environment variables",systemEnvPropCount);
+        try {
+            for (final Path path : propertiesPaths) {
+                if (Files.exists(path)) {
+                    final AtomicInteger propsLoaded = new AtomicInteger(0);
+                    try (final InputStream inStream = new BufferedInputStream(new FileInputStream(path.toFile()))) {
+                        Properties properties = new Properties();
+                        properties.load(inStream);
+                        properties.entrySet().stream().forEach((entry) -> {
+                            final VariableDescriptor desc = new VariableDescriptor.Builder(entry.getKey().toString())
+                                    .description(path.toString())
+                                    .sensitive(false)
+                                    .build();
+                            newMap.put(desc, entry.getValue().toString());
+                            propsLoaded.incrementAndGet();
+                        });
+                    }
+                    totalPropertiesLoaded += propsLoaded.get();
+                    if(propsLoaded.get() > 0){
+                        LOG.info("Loaded {} properties from '{}'", propsLoaded.get(), path);
+                    }else{
+                        LOG.warn("No properties loaded from '{}'", path);
+                    }
+                } else {
+                    LOG.warn("Skipping property file {} as it does not appear to exist", path);
+                }
+            }
+        } catch (final IOException ioe) {
+            LOG.error("Unable to complete variable registry loading from files due to ", ioe);
+        }
+
+        LOG.info("Loaded a total of {} properties.  Including precedence overrides effective accessible registry key size is {}", totalPropertiesLoaded, newMap.size());
+        map = Collections.unmodifiableMap(newMap);
+    }
+
+    @Override
+    public Map<VariableDescriptor, String> getVariableMap() {
+        return map;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/variable/MutableVariableRegistry.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/variable/MutableVariableRegistry.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/variable/MutableVariableRegistry.java
new file mode 100644
index 0000000..e37b402
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/variable/MutableVariableRegistry.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.registry.variable;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.nifi.registry.VariableDescriptor;
+import org.apache.nifi.registry.VariableRegistry;
+
+public class MutableVariableRegistry extends StandardComponentVariableRegistry implements VariableRegistry {
+    private volatile Map<VariableDescriptor, String> variableMap = new HashMap<>();
+
+    public MutableVariableRegistry(final VariableRegistry parent) {
+        super(parent);
+    }
+
+    @Override
+    public Map<VariableDescriptor, String> getVariableMap() {
+        return variableMap;
+    }
+
+    public void setVariables(final Map<VariableDescriptor, String> variables) {
+        final Map<VariableDescriptor, String> curVariableMap = this.variableMap;
+        final Map<VariableDescriptor, String> updatedVariableMap = new HashMap<>(curVariableMap);
+        for (final Map.Entry<VariableDescriptor, String> entry : variables.entrySet()) {
+            if (entry.getValue() == null) {
+                updatedVariableMap.remove(entry.getKey());
+            } else {
+                updatedVariableMap.put(entry.getKey(), entry.getValue());
+            }
+        }
+
+        this.variableMap = Collections.unmodifiableMap(updatedVariableMap);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/variable/StandardComponentVariableRegistry.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/variable/StandardComponentVariableRegistry.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/variable/StandardComponentVariableRegistry.java
new file mode 100644
index 0000000..b3c31f3
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/variable/StandardComponentVariableRegistry.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.registry.variable;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.nifi.registry.ComponentVariableRegistry;
+import org.apache.nifi.registry.VariableDescriptor;
+import org.apache.nifi.registry.VariableRegistry;
+
+public class StandardComponentVariableRegistry implements ComponentVariableRegistry {
+    private volatile VariableRegistry parent;
+
+    public StandardComponentVariableRegistry(final VariableRegistry parent) {
+        this.parent = Objects.requireNonNull(parent);
+    }
+
+    @Override
+    public Map<VariableDescriptor, String> getVariableMap() {
+        return Collections.emptyMap();
+    }
+
+    @Override
+    public VariableRegistry getParent() {
+        return parent;
+    }
+
+    @Override
+    public void setParent(final VariableRegistry parentRegistry) {
+        this.parent = parentRegistry;
+    }
+
+    @Override
+    public VariableDescriptor getVariableKey(final String name) {
+        if (name == null) {
+            return null;
+        }
+
+        final VariableDescriptor spec = new VariableDescriptor(name);
+        for (final Map.Entry<VariableDescriptor, String> entry : getVariableMap().entrySet()) {
+            if (entry.getKey().equals(spec)) {
+                return entry.getKey();
+            }
+        }
+
+        return null;
+    }
+
+    @Override
+    public String getVariableValue(final String name) {
+        if (name == null) {
+            return null;
+        }
+
+        final VariableDescriptor descriptor = new VariableDescriptor(name);
+        final String value = getVariableMap().get(descriptor);
+        if (value != null) {
+            return value;
+        }
+
+        return parent.getVariableValue(descriptor);
+    }
+
+    @Override
+    public String getVariableValue(final VariableDescriptor descriptor) {
+        if (descriptor == null) {
+            return null;
+        }
+
+        final String value = getVariableMap().get(descriptor);
+        if (value != null) {
+            return value;
+        }
+
+        return parent.getVariableValue(descriptor);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/FileBasedVariableRegistry.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/FileBasedVariableRegistry.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/FileBasedVariableRegistry.java
deleted file mode 100644
index c4079a7..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/FileBasedVariableRegistry.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.util;
-
-import java.io.BufferedInputStream;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Properties;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A file based variable registry that loads all properties from files specified
- * during construction and is backed by system properties and environment
- * variables accessible to the JVM.
- */
-public class FileBasedVariableRegistry implements VariableRegistry {
-
-    private final static Logger LOG = LoggerFactory.getLogger(FileBasedVariableRegistry.class);
-    final Map<VariableDescriptor, String> map;
-
-    public FileBasedVariableRegistry(final Path[] propertiesPaths) {
-        final Map<VariableDescriptor, String> newMap = new HashMap<>(VariableRegistry.ENVIRONMENT_SYSTEM_REGISTRY.getVariableMap());
-        final int systemEnvPropCount = newMap.size();
-        int totalPropertiesLoaded = systemEnvPropCount;
-        LOG.info("Loaded {} properties from system properties and environment variables",systemEnvPropCount);
-        try {
-            for (final Path path : propertiesPaths) {
-                if (Files.exists(path)) {
-                    final AtomicInteger propsLoaded = new AtomicInteger(0);
-                    try (final InputStream inStream = new BufferedInputStream(new FileInputStream(path.toFile()))) {
-                        Properties properties = new Properties();
-                        properties.load(inStream);
-                        properties.entrySet().stream().forEach((entry) -> {
-                            final VariableDescriptor desc = new VariableDescriptor.Builder(entry.getKey().toString())
-                                    .description(path.toString())
-                                    .sensitive(false)
-                                    .build();
-                            newMap.put(desc, entry.getValue().toString());
-                            propsLoaded.incrementAndGet();
-                        });
-                    }
-                    totalPropertiesLoaded += propsLoaded.get();
-                    if(propsLoaded.get() > 0){
-                        LOG.info("Loaded {} properties from '{}'", propsLoaded.get(), path);
-                    }else{
-                        LOG.warn("No properties loaded from '{}'", path);
-                    }
-                } else {
-                    LOG.warn("Skipping property file {} as it does not appear to exist", path);
-                }
-            }
-        } catch (final IOException ioe) {
-            LOG.error("Unable to complete variable registry loading from files due to ", ioe);
-        }
-        LOG.info("Loaded a total of {} properties.  Including precedence overrides effective accessible registry key size is {}", totalPropertiesLoaded, newMap.size());
-        map = newMap;
-    }
-
-    @Override
-    public Map<VariableDescriptor, String> getVariableMap() {
-        return Collections.unmodifiableMap(map);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd
index 30fff1f..247a790 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd
@@ -147,9 +147,15 @@
             <xs:element name="connection" type="ConnectionType" minOccurs="0" maxOccurs="unbounded" />
             <xs:element name="controllerService" type="ControllerServiceType" minOccurs="0" maxOccurs="unbounded" />
             <xs:element name="template" type="TemplateType" minOccurs="0" maxOccurs="unbounded" />
+            <xs:element name="variable" type="VariableType" minOccurs="0" maxOccurs="unbounded"  />
         </xs:sequence>
     </xs:complexType>
     
+    <xs:complexType name="VariableType">
+        <xs:attribute name="name" />
+        <xs:attribute name="value" />
+    </xs:complexType>
+    
     <!-- Same as ProcessGroupType except that instead of input ports & output ports being of type PortType,
     they are of type RootGroupPortType -->
     <xs:complexType name="RootProcessGroupType">

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml
index bf3fd2a..b3c426c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml
@@ -23,7 +23,7 @@
     </bean>
 
     <!-- variable registry -->
-    <bean id="variableRegistry" class="org.apache.nifi.util.FileBasedVariableRegistry">
+    <bean id="variableRegistry" class="org.apache.nifi.registry.variable.FileBasedVariableRegistry">
         <constructor-arg type="java.nio.file.Path[]" value="#{nifiProperties.getVariableRegistryPropertiesPaths()}" />
     </bean>
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
index 1957cf4..6973d12 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
@@ -28,7 +28,7 @@ import org.apache.nifi.controller.serialization.StandardFlowSerializer;
 import org.apache.nifi.encrypt.StringEncryptor;
 import org.apache.nifi.events.VolatileBulletinRepository;
 import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.util.FileBasedVariableRegistry;
+import org.apache.nifi.registry.variable.FileBasedVariableRegistry;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.api.dto.ConnectableDTO;
 import org.apache.nifi.web.api.dto.ConnectionDTO;

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java
index 691af10..596c00f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java
@@ -49,9 +49,9 @@ import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.provenance.MockProvenanceRepository;
 import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.variable.FileBasedVariableRegistry;
 import org.apache.nifi.reporting.BulletinRepository;
 import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.util.FileBasedVariableRegistry;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.api.dto.BundleDTO;
 import org.apache.nifi.web.api.dto.ControllerServiceDTO;

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java
index 32b6f53..33c33c9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java
@@ -17,6 +17,26 @@
 
 package org.apache.nifi.controller;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
@@ -46,6 +66,7 @@ import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.registry.VariableDescriptor;
 import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.variable.StandardComponentVariableRegistry;
 import org.apache.nifi.test.processors.ModifiesClasspathNoAnnotationProcessor;
 import org.apache.nifi.test.processors.ModifiesClasspathProcessor;
 import org.apache.nifi.util.MockPropertyValue;
@@ -56,26 +77,6 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
-import java.io.File;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.Callable;
-import java.util.concurrent.Future;
-import java.util.concurrent.ScheduledExecutorService;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 public class TestStandardProcessorNode {
 
     private MockVariableRegistry variableRegistry;
@@ -98,10 +99,10 @@ public class TestStandardProcessorNode {
 
         final LoggableComponent<Processor> loggableComponent = new LoggableComponent<>(processor, coordinate, null);
         final StandardProcessorNode procNode = new StandardProcessorNode(loggableComponent, uuid, createValidationContextFactory(), null, null,
-                NiFiProperties.createBasicNiFiProperties(null, null), VariableRegistry.EMPTY_REGISTRY, reloadComponent);
+            NiFiProperties.createBasicNiFiProperties(null, null), new StandardComponentVariableRegistry(VariableRegistry.EMPTY_REGISTRY), reloadComponent);
         final ScheduledExecutorService taskScheduler = new FlowEngine(2, "TestClasspathResources", true);
 
-        final StandardProcessContext processContext = new StandardProcessContext(procNode, null, null, null, null);
+        final StandardProcessContext processContext = new StandardProcessContext(procNode, null, null, null);
         final SchedulingAgentCallback schedulingAgentCallback = new SchedulingAgentCallback() {
             @Override
             public void postMonitor() {
@@ -414,7 +415,8 @@ public class TestStandardProcessorNode {
         processor.initialize(initContext);
 
         final LoggableComponent<Processor> loggableComponent = new LoggableComponent<>(processor, systemBundle.getBundleDetails().getCoordinate(), componentLog);
-        return new StandardProcessorNode(loggableComponent, uuid, validationContextFactory, processScheduler, null, niFiProperties, variableRegistry, reloadComponent);
+        return new StandardProcessorNode(loggableComponent, uuid, validationContextFactory, processScheduler,
+            null, niFiProperties, new StandardComponentVariableRegistry(variableRegistry), reloadComponent);
     }
 
     private static class MockReloadComponent implements ReloadComponent {

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/reporting/TestStandardReportingContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/reporting/TestStandardReportingContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/reporting/TestStandardReportingContext.java
index 1903179..1dc74ab 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/reporting/TestStandardReportingContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/reporting/TestStandardReportingContext.java
@@ -16,15 +16,6 @@
  */
 package org.apache.nifi.controller.reporting;
 
-import static org.junit.Assert.assertEquals;
-
-import java.io.File;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashSet;
-import java.util.Map;
-import java.util.Set;
-
 import org.apache.commons.io.FileUtils;
 import org.apache.nifi.admin.service.AuditService;
 import org.apache.nifi.authorization.AbstractPolicyBasedAuthorizer;
@@ -45,14 +36,23 @@ import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.provenance.MockProvenanceRepository;
 import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.variable.FileBasedVariableRegistry;
 import org.apache.nifi.reporting.BulletinRepository;
-import org.apache.nifi.util.FileBasedVariableRegistry;
 import org.apache.nifi.util.NiFiProperties;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import java.io.File;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+
 public class TestStandardReportingContext {
 
     private FlowController controller;

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java
index 7f36a65..c544ef4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java
@@ -46,7 +46,7 @@ import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.provenance.MockProvenanceRepository;
-import org.apache.nifi.util.FileBasedVariableRegistry;
+import org.apache.nifi.registry.variable.FileBasedVariableRegistry;
 import org.apache.nifi.util.NiFiProperties;
 import org.junit.After;
 import org.junit.Before;

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java
index dfd627f..2c59964 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java
@@ -74,6 +74,7 @@ import org.apache.nifi.processor.StandardProcessorInitializationContext;
 import org.apache.nifi.processor.StandardValidationContextFactory;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.variable.StandardComponentVariableRegistry;
 import org.apache.nifi.reporting.AbstractReportingTask;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.reporting.ReportingContext;
@@ -110,7 +111,7 @@ public class TestStandardProcessScheduler {
         systemBundle = SystemBundle.create(nifiProperties);
         ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
 
-        scheduler = new StandardProcessScheduler(Mockito.mock(ControllerServiceProvider.class), null, stateMgrProvider, variableRegistry, nifiProperties);
+        scheduler = new StandardProcessScheduler(Mockito.mock(ControllerServiceProvider.class), null, stateMgrProvider, nifiProperties);
         scheduler.setSchedulingAgent(SchedulingStrategy.TIMER_DRIVEN, Mockito.mock(SchedulingAgent.class));
 
         reportingTask = new TestReportingTask();
@@ -122,7 +123,8 @@ public class TestStandardProcessScheduler {
         final ComponentLog logger = Mockito.mock(ComponentLog.class);
         final ReloadComponent reloadComponent = Mockito.mock(ReloadComponent.class);
         final LoggableComponent<ReportingTask> loggableComponent = new LoggableComponent<>(reportingTask, systemBundle.getBundleDetails().getCoordinate(), logger);
-        taskNode = new StandardReportingTaskNode(loggableComponent, UUID.randomUUID().toString(), null, scheduler, validationContextFactory, variableRegistry, reloadComponent);
+        taskNode = new StandardReportingTaskNode(loggableComponent, UUID.randomUUID().toString(), null, scheduler, validationContextFactory,
+            new StandardComponentVariableRegistry(variableRegistry), reloadComponent);
 
         controller = Mockito.mock(FlowController.class);
 
@@ -196,7 +198,7 @@ public class TestStandardProcessScheduler {
         final LoggableComponent<Processor> loggableComponent = new LoggableComponent<>(proc, systemBundle.getBundleDetails().getCoordinate(), null);
         final ProcessorNode procNode = new StandardProcessorNode(loggableComponent, uuid,
                 new StandardValidationContextFactory(serviceProvider, variableRegistry),
-                scheduler, serviceProvider, nifiProperties, VariableRegistry.EMPTY_REGISTRY, reloadComponent);
+                scheduler, serviceProvider, nifiProperties, new StandardComponentVariableRegistry(VariableRegistry.EMPTY_REGISTRY), reloadComponent);
         rootGroup.addProcessor(procNode);
 
         Map<String, String> procProps = new HashMap<>();
@@ -580,6 +582,6 @@ public class TestStandardProcessScheduler {
     }
 
     private ProcessScheduler createScheduler() {
-        return new StandardProcessScheduler(null, null, stateMgrProvider, variableRegistry, nifiProperties);
+        return new StandardProcessScheduler(null, null, stateMgrProvider, nifiProperties);
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/serialization/StandardFlowSerializerTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/serialization/StandardFlowSerializerTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/serialization/StandardFlowSerializerTest.java
index c235a57..7a49103 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/serialization/StandardFlowSerializerTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/serialization/StandardFlowSerializerTest.java
@@ -29,8 +29,8 @@ import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.provenance.MockProvenanceRepository;
 import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.variable.FileBasedVariableRegistry;
 import org.apache.nifi.reporting.BulletinRepository;
-import org.apache.nifi.util.FileBasedVariableRegistry;
 import org.apache.nifi.util.NiFiProperties;
 import org.junit.Before;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java
index 4a97b8a..55263f8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java
@@ -24,8 +24,8 @@ import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.NarClassLoaders;
 import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.variable.FileBasedVariableRegistry;
 import org.apache.nifi.reporting.InitializationException;
-import org.apache.nifi.util.FileBasedVariableRegistry;
 import org.apache.nifi.util.NiFiProperties;
 import org.junit.Before;
 import org.junit.BeforeClass;

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java
index e82085e..0d15143 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java
@@ -17,6 +17,22 @@
  */
 package org.apache.nifi.controller.service;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.beans.PropertyDescriptor;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.components.state.StateManager;
 import org.apache.nifi.components.state.StateManagerProvider;
@@ -40,6 +56,8 @@ import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.processor.Processor;
 import org.apache.nifi.processor.StandardValidationContextFactory;
 import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.variable.MutableVariableRegistry;
+import org.apache.nifi.registry.variable.StandardComponentVariableRegistry;
 import org.apache.nifi.util.NiFiProperties;
 import org.junit.Assert;
 import org.junit.Before;
@@ -49,22 +67,6 @@ import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
-import java.beans.PropertyDescriptor;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
 public class TestStandardControllerServiceProvider {
 
     private static StateManagerProvider stateManagerProvider = new StateManagerProvider() {
@@ -129,7 +131,7 @@ public class TestStandardControllerServiceProvider {
     }
 
     private StandardProcessScheduler createScheduler() {
-        return new StandardProcessScheduler(null, null, stateManagerProvider, variableRegistry, niFiProperties);
+        return new StandardProcessScheduler(null, null, stateManagerProvider, niFiProperties);
     }
 
     private void setProperty(ControllerServiceNode serviceNode, String propName, String propValue) {
@@ -432,9 +434,10 @@ public class TestStandardControllerServiceProvider {
         final LoggableComponent<Processor> dummyProcessor = new LoggableComponent<>(new DummyProcessor(), systemBundle.getBundleDetails().getCoordinate(), null);
         final ProcessorNode procNode = new StandardProcessorNode(dummyProcessor, UUID.randomUUID().toString(),
                 new StandardValidationContextFactory(serviceProvider, null), scheduler, serviceProvider, niFiProperties,
-                VariableRegistry.EMPTY_REGISTRY, reloadComponent);
+                new StandardComponentVariableRegistry(VariableRegistry.EMPTY_REGISTRY), reloadComponent);
 
-        final ProcessGroup group = new StandardProcessGroup(UUID.randomUUID().toString(), serviceProvider, scheduler, null, null, Mockito.mock(FlowController.class), variableRegistry);
+        final ProcessGroup group = new StandardProcessGroup(UUID.randomUUID().toString(), serviceProvider, scheduler, null, null, Mockito.mock(FlowController.class),
+            new MutableVariableRegistry(variableRegistry));
         group.addProcessor(procNode);
         procNode.setProcessGroup(group);
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java
index b6f70a1..9725ed8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java
@@ -18,11 +18,13 @@
 package org.apache.nifi.controller.service.mock;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.CompletableFuture;
 
 import org.apache.nifi.authorization.Resource;
 import org.apache.nifi.authorization.resource.Authorizable;
@@ -32,6 +34,7 @@ import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.connectable.Position;
 import org.apache.nifi.connectable.Positionable;
+import org.apache.nifi.controller.ConfiguredComponent;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
 import org.apache.nifi.controller.Snippet;
@@ -41,12 +44,15 @@ import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.ProcessGroupCounts;
 import org.apache.nifi.groups.RemoteProcessGroup;
+import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.variable.MutableVariableRegistry;
 import org.apache.nifi.remote.RemoteGroupPort;
 
 public class MockProcessGroup implements ProcessGroup {
     private final Map<String, ControllerServiceNode> serviceMap = new HashMap<>();
     private final Map<String, ProcessorNode> processorMap = new HashMap<>();
     private final FlowController flowController;
+    private final MutableVariableRegistry variableRegistry = new MutableVariableRegistry(VariableRegistry.ENVIRONMENT_SYSTEM_REGISTRY);
 
     public MockProcessGroup(final FlowController flowController) {
         this.flowController = flowController;
@@ -143,8 +149,8 @@ public class MockProcessGroup implements ProcessGroup {
     }
 
     @Override
-    public void startProcessor(final ProcessorNode processor) {
-
+    public CompletableFuture<Void> startProcessor(final ProcessorNode processor) {
+        return CompletableFuture.completedFuture(null);
     }
 
     @Override
@@ -163,8 +169,8 @@ public class MockProcessGroup implements ProcessGroup {
     }
 
     @Override
-    public void stopProcessor(final ProcessorNode processor) {
-
+    public CompletableFuture<Void> stopProcessor(final ProcessorNode processor) {
+        return CompletableFuture.completedFuture(null);
     }
 
     @Override
@@ -601,4 +607,22 @@ public class MockProcessGroup implements ProcessGroup {
     @Override
     public void verifyCanStop(final Connectable connectable) {
     }
+
+    @Override
+    public MutableVariableRegistry getVariableRegistry() {
+        return variableRegistry;
+    }
+
+    @Override
+    public void verifyCanUpdateVariables(Map<String, String> updatedVariables) {
+    }
+
+    @Override
+    public void setVariables(Map<String, String> variables) {
+    }
+
+    @Override
+    public Set<ConfiguredComponent> getComponentsAffectedByVariable(String variableName) {
+        return Collections.emptySet();
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/util/TestFileBasedVariableRegistry.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/util/TestFileBasedVariableRegistry.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/util/TestFileBasedVariableRegistry.java
index cf44435..daed0e0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/util/TestFileBasedVariableRegistry.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/util/TestFileBasedVariableRegistry.java
@@ -21,6 +21,7 @@ import java.nio.file.Paths;
 import java.util.Map;
 import org.apache.nifi.registry.VariableDescriptor;
 import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.variable.FileBasedVariableRegistry;
 import org.junit.Test;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertEquals;

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessGroupAuditor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessGroupAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessGroupAuditor.java
index 391f9e2..22ad122 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessGroupAuditor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessGroupAuditor.java
@@ -26,6 +26,7 @@ import org.apache.nifi.action.details.FlowChangeMoveDetails;
 import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.authorization.user.NiFiUserUtils;
 import org.apache.nifi.controller.ScheduledState;
+import org.apache.nifi.controller.service.ControllerServiceState;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.web.api.dto.ProcessGroupDTO;
 import org.apache.nifi.web.dao.ProcessGroupDAO;
@@ -175,6 +176,60 @@ public class ProcessGroupAuditor extends NiFiAuditor {
         + "execution(void scheduleComponents(java.lang.String, org.apache.nifi.controller.ScheduledState, java.util.Set)) && "
         + "args(groupId, state)")
     public void scheduleComponentsAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, ScheduledState state) throws Throwable {
+        final Operation operation;
+        // determine the running state
+        if (ScheduledState.RUNNING.equals(state)) {
+            operation = Operation.Start;
+        } else {
+            operation = Operation.Stop;
+        }
+
+        saveUpdateAction(proceedingJoinPoint, groupId, operation);
+    }
+
+
+    /**
+     * Audits the update of controller serivce state
+     *
+     * @param proceedingJoinPoint join point
+     * @param groupId group id
+     * @param state controller serivce state state
+     * @throws Throwable ex
+     */
+    @Around("within(org.apache.nifi.web.dao.ProcessGroupDAO+) && "
+        + "execution(java.util.concurrent.Future activateControllerServices(java.lang.String, org.apache.nifi.controller.service.ControllerServiceState, java.util.Set)) && "
+        + "args(groupId, state)")
+    public void activateControllerServicesAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, ControllerServiceState state) throws Throwable {
+
+        // determine the service state
+        final Operation operation;
+        if (ControllerServiceState.ENABLED.equals(state)) {
+            operation = Operation.Enable;
+        } else {
+            operation = Operation.Disable;
+        }
+
+        saveUpdateAction(proceedingJoinPoint, groupId, operation);
+    }
+
+    /**
+     * Audits the update of process group variable registry.
+     *
+     * @param proceedingJoinPoint join point
+     * @param groupId group id
+     * @throws Throwable ex
+     */
+    @Around("within(org.apache.nifi.web.dao.ProcessGroupDAO+) && "
+        + "execution(org.apache.nifi.groups.ProcessGroup updateVariableRegistry(org.apache.nifi.web.api.dto.VariableRegistryDTO)) && "
+        + "args(groupId)")
+    public void updateVariableRegistryAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId) throws Throwable {
+        final Operation operation = Operation.Configure;
+        saveUpdateAction(proceedingJoinPoint, groupId, operation);
+    }
+
+
+
+    private void saveUpdateAction(final ProceedingJoinPoint proceedingJoinPoint, final String groupId, final Operation operation) throws Throwable {
         ProcessGroupDAO processGroupDAO = getProcessGroupDAO();
         ProcessGroup processGroup = processGroupDAO.getProcessGroup(groupId);
 
@@ -191,13 +246,7 @@ public class ProcessGroupAuditor extends NiFiAuditor {
         action.setSourceName(processGroup.getName());
         action.setSourceType(Component.ProcessGroup);
         action.setTimestamp(new Date());
-
-        // determine the running state
-        if (ScheduledState.RUNNING.equals(state)) {
-            action.setOperation(Operation.Start);
-        } else {
-            action.setOperation(Operation.Stop);
-        }
+        action.setOperation(operation);
 
         // add this action
         saveAction(action, logger);

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/registry/variable/VariableRegistryUpdateRequest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/registry/variable/VariableRegistryUpdateRequest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/registry/variable/VariableRegistryUpdateRequest.java
new file mode 100644
index 0000000..82d4683
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/registry/variable/VariableRegistryUpdateRequest.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.registry.variable;
+
+import java.util.Date;
+import java.util.concurrent.atomic.AtomicReference;
+
+public class VariableRegistryUpdateRequest {
+    private final String requestId;
+    private final String processGroupId;
+    private volatile Date submissionTime = new Date();
+    private volatile Date lastUpdated = new Date();
+    private volatile boolean complete = false;
+
+    private final AtomicReference<String> failureReason = new AtomicReference<>();
+
+    private final VariableRegistryUpdateStep identifyComponentsStep = new VariableRegistryUpdateStep("Identifying components affected");
+    private final VariableRegistryUpdateStep stopProcessors = new VariableRegistryUpdateStep("Stopping affected Processors");
+    private final VariableRegistryUpdateStep disableServices = new VariableRegistryUpdateStep("Disabling affected Controller Services");
+    private final VariableRegistryUpdateStep applyUpdates = new VariableRegistryUpdateStep("Applying Updates");
+    private final VariableRegistryUpdateStep enableServices = new VariableRegistryUpdateStep("Re-Enabling affected Controller Services");
+    private final VariableRegistryUpdateStep startProcessors = new VariableRegistryUpdateStep("Restarting affected Processors");
+
+    public VariableRegistryUpdateRequest(final String requestId, final String processGroupId) {
+        this.requestId = requestId;
+        this.processGroupId = processGroupId;
+    }
+
+    public String getProcessGroupId() {
+        return processGroupId;
+    }
+
+
+    public String getRequestId() {
+        return requestId;
+    }
+
+    public Date getSubmissionTime() {
+        return submissionTime;
+    }
+
+    public Date getLastUpdated() {
+        return lastUpdated;
+    }
+
+    public void setLastUpdated(Date lastUpdated) {
+        this.lastUpdated = lastUpdated;
+    }
+
+    public VariableRegistryUpdateStep getIdentifyRelevantComponentsStep() {
+        return identifyComponentsStep;
+    }
+
+    public VariableRegistryUpdateStep getStopProcessorsStep() {
+        return stopProcessors;
+    }
+
+    public VariableRegistryUpdateStep getDisableServicesStep() {
+        return disableServices;
+    }
+
+    public VariableRegistryUpdateStep getApplyUpdatesStep() {
+        return applyUpdates;
+    }
+
+    public VariableRegistryUpdateStep getEnableServicesStep() {
+        return enableServices;
+    }
+
+    public VariableRegistryUpdateStep getStartProcessorsStep() {
+        return startProcessors;
+    }
+
+    public boolean isComplete() {
+        return complete;
+    }
+
+    public void setComplete(boolean complete) {
+        this.complete = complete;
+    }
+
+    public String getFailureReason() {
+        return failureReason.get();
+    }
+
+    public void setFailureReason(String reason) {
+        this.failureReason.set(reason);
+    }
+
+    public void cancel() {
+        this.failureReason.compareAndSet(null, "Update was cancelled");
+        this.complete = true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/registry/variable/VariableRegistryUpdateStep.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/registry/variable/VariableRegistryUpdateStep.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/registry/variable/VariableRegistryUpdateStep.java
new file mode 100644
index 0000000..6013795
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/registry/variable/VariableRegistryUpdateStep.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.registry.variable;
+
+public class VariableRegistryUpdateStep {
+    private final String description;
+    private volatile boolean complete;
+    private volatile String failureReason;
+
+    public VariableRegistryUpdateStep(final String description) {
+        this.description = description;
+    }
+
+    public boolean isComplete() {
+        return complete;
+    }
+
+    public void setComplete(boolean complete) {
+        this.complete = complete;
+    }
+
+    public String getFailureReason() {
+        return failureReason;
+    }
+
+    public void setFailureReason(String failureReason) {
+        this.failureReason = failureReason;
+    }
+
+    public String getDescription() {
+        return description;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
index 6f9ea98..6fed58e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
@@ -16,6 +16,13 @@
  */
 package org.apache.nifi.web;
 
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.function.Function;
+
 import org.apache.nifi.authorization.AuthorizeAccess;
 import org.apache.nifi.authorization.RequestAction;
 import org.apache.nifi.authorization.user.NiFiUser;
@@ -24,6 +31,7 @@ import org.apache.nifi.controller.repository.claim.ContentDirection;
 import org.apache.nifi.controller.service.ControllerServiceState;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.web.api.dto.AccessPolicyDTO;
+import org.apache.nifi.web.api.dto.AffectedComponentDTO;
 import org.apache.nifi.web.api.dto.BulletinBoardDTO;
 import org.apache.nifi.web.api.dto.BulletinDTO;
 import org.apache.nifi.web.api.dto.BulletinQueryDTO;
@@ -57,6 +65,7 @@ import org.apache.nifi.web.api.dto.SystemDiagnosticsDTO;
 import org.apache.nifi.web.api.dto.TemplateDTO;
 import org.apache.nifi.web.api.dto.UserDTO;
 import org.apache.nifi.web.api.dto.UserGroupDTO;
+import org.apache.nifi.web.api.dto.VariableRegistryDTO;
 import org.apache.nifi.web.api.dto.action.HistoryDTO;
 import org.apache.nifi.web.api.dto.action.HistoryQueryDTO;
 import org.apache.nifi.web.api.dto.provenance.ProvenanceDTO;
@@ -67,6 +76,7 @@ import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
 import org.apache.nifi.web.api.dto.status.ControllerStatusDTO;
 import org.apache.nifi.web.api.entity.AccessPolicyEntity;
 import org.apache.nifi.web.api.entity.ActionEntity;
+import org.apache.nifi.web.api.entity.ActivateControllerServicesEntity;
 import org.apache.nifi.web.api.entity.BulletinEntity;
 import org.apache.nifi.web.api.entity.ConnectionEntity;
 import org.apache.nifi.web.api.entity.ConnectionStatusEntity;
@@ -96,13 +106,7 @@ import org.apache.nifi.web.api.entity.StatusHistoryEntity;
 import org.apache.nifi.web.api.entity.TemplateEntity;
 import org.apache.nifi.web.api.entity.UserEntity;
 import org.apache.nifi.web.api.entity.UserGroupEntity;
-
-import java.util.Date;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-import java.util.function.Function;
+import org.apache.nifi.web.api.entity.VariableRegistryEntity;
 
 /**
  * Defines the NiFiServiceFacade interface.
@@ -887,6 +891,48 @@ public interface NiFiServiceFacade {
     ProcessGroupEntity getProcessGroup(String groupId);
 
     /**
+     * Returns the Variable Registry for the Process Group with the given ID
+     *
+     * @param groupId the ID of the Process Group
+     * @param includeAncestorGroups whether or not to include the variables that are defined in the the process group's parent group & its parent group, etc.
+     * @return the Variable Registry transfer object
+     */
+    VariableRegistryEntity getVariableRegistry(String groupId, boolean includeAncestorGroups);
+
+    /**
+     * Returns a Variable Registry that includes the variables in the given DTO but has the affected components populated
+     *
+     * @param variableRegistryDto the Variable Registry that contains the variables of interest
+     * @return a Variable Registry that has the affected components populated
+     */
+    VariableRegistryEntity populateAffectedComponents(VariableRegistryDTO variableRegistryDto);
+
+    /**
+     * Updates the variable registry on behalf of the user currently logged in
+     *
+     * @param revision Revision to compare with current base revision
+     * @param variableRegistryDto the Variable Registry
+     */
+    VariableRegistryEntity updateVariableRegistry(Revision revision, VariableRegistryDTO variableRegistryDto);
+
+    /**
+     * Updates the variable registry on behalf of the given user
+     *
+     * @param user the user who performed the action
+     * @param revision Revision to compare with current base revision
+     * @param variableRegistryDto the Variable Registry
+     */
+    VariableRegistryEntity updateVariableRegistry(NiFiUser user, Revision revision, VariableRegistryDTO variableRegistryDto);
+
+    /**
+     * Determines which components will be affected by updating the given Variable Registry
+     *
+     * @param variableRegistryDto the variable registry
+     * @return the components that will be affected
+     */
+    Set<AffectedComponentDTO> getComponentsAffectedByVariableRegistryUpdate(VariableRegistryDTO variableRegistryDto);
+
+    /**
      * Gets all process groups in the specified parent group.
      *
      * @param parentGroupId The id of the parent group
@@ -904,7 +950,37 @@ public interface NiFiServiceFacade {
     void verifyScheduleComponents(String processGroupId, ScheduledState state, Set<String> componentIds);
 
     /**
-     * Schedules all applicable components under the specified ProcessGroup.
+     * Verifies the controller services with the given ID's can be enabled or disabled
+     *
+     * @param processGroupId the ID of the process group
+     * @param state the state
+     * @param serviceIds the id's of the services
+     */
+    void verifyActivateControllerServices(String processGroupId, ControllerServiceState state, Set<String> serviceIds);
+
+    /**
+     * Enables or disables the controller services with the given IDs & Revisions on behalf of the currently logged in user
+     *
+     * @param processGroupId the ID of the process group
+     * @param state the desired state of the services
+     * @param serviceRevisions a mapping of Controller Service ID to current Revision
+     * @return snapshot
+     */
+    ActivateControllerServicesEntity activateControllerServices(String processGroupId, ControllerServiceState state, Map<String, Revision> serviceRevisions);
+
+    /**
+     * Enables or disables the controller services with the given IDs & Revisions on behalf of the given user
+     *
+     * @param user the user performing the action
+     * @param processGroupId the ID of the process group
+     * @param state the desired state of the services
+     * @param serviceRevisions a mapping of Controller Service ID to current Revision
+     * @return snapshot
+     */
+    ActivateControllerServicesEntity activateControllerServices(NiFiUser user, String processGroupId, ControllerServiceState state, Map<String, Revision> serviceRevisions);
+
+    /**
+     * Schedules all applicable components under the specified ProcessGroup on behalf of the currently logged in user.
      *
      * @param processGroupId The ProcessGroup id
      * @param state schedule state
@@ -914,6 +990,17 @@ public interface NiFiServiceFacade {
     ScheduleComponentsEntity scheduleComponents(String processGroupId, ScheduledState state, Map<String, Revision> componentRevisions);
 
     /**
+     * Schedules all applicable components under the specified ProcessGroup on behalf of the given user.
+     *
+     * @param user the user performing the action
+     * @param processGroupId The ProcessGroup id
+     * @param state schedule state
+     * @param componentRevisions components and their revision
+     * @return snapshot
+     */
+    ScheduleComponentsEntity scheduleComponents(NiFiUser user, String processGroupId, ScheduledState state, Map<String, Revision> componentRevisions);
+
+    /**
      * Updates the specified process group.
      *
      * @param revision Revision to compare with current base revision
@@ -923,6 +1010,14 @@ public interface NiFiServiceFacade {
     ProcessGroupEntity updateProcessGroup(Revision revision, ProcessGroupDTO processGroupDTO);
 
     /**
+     * Verifies that the Process Group identified by the given DTO can be updated in the manner appropriate according
+     * to the DTO
+     *
+     * @param processGroupDTO the DTO that indicates the updates to occur
+     */
+    void verifyUpdateProcessGroup(ProcessGroupDTO processGroupDTO);
+
+    /**
      * Verifies the specified process group can be removed.
      *
      * @param groupId The id of the process group
@@ -1378,9 +1473,22 @@ public interface NiFiServiceFacade {
      * Gets all controller services that belong to the given group and its parent/ancestor groups
      *
      * @param groupId the id of the process group of interest
+     * @param includeAncestorGroups if true, parent and ancestor groups' services will be returned as well
+     * @param includeDescendantGroups if true, child and descendant groups' services will be returned as well
+     * @return services
+     */
+    Set<ControllerServiceEntity> getControllerServices(String groupId, boolean includeAncestorGroups, boolean includeDescendantGroups);
+
+    /**
+     * Gets all controller services that belong to the given group and its parent/ancestor groups
+     *
+     * @param groupId the id of the process group of interest
+     * @param includeAncestorGroups if true, parent and ancestor groups' services will be returned as well
+     * @param includeDescendantGroups if true, child and descendant groups' services will be returned as well
+     * @param user the user that is retrieving the Controller Services
      * @return services
      */
-    Set<ControllerServiceEntity> getControllerServices(String groupId);
+    Set<ControllerServiceEntity> getControllerServices(String groupId, boolean includeAncestorGroups, boolean includeDescendantGroups, NiFiUser user);
 
     /**
      * Gets the specified controller service.

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacadeLock.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacadeLock.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacadeLock.java
index 11573a5..d0230db 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacadeLock.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacadeLock.java
@@ -103,6 +103,18 @@ public class NiFiServiceFacadeLock {
     }
 
     @Around("within(org.apache.nifi.web.NiFiServiceFacade+) && "
+        + "execution(* activate*(..))")
+    public Object activateLock(ProceedingJoinPoint proceedingJoinPoint) throws Throwable {
+        return proceedWithWriteLock(proceedingJoinPoint);
+    }
+
+    @Around("within(org.apache.nifi.web.NiFiServiceFacade+) && "
+        + "execution(* populate*(..))")
+    public Object populateLock(ProceedingJoinPoint proceedingJoinPoint) throws Throwable {
+        return proceedWithWriteLock(proceedingJoinPoint);
+    }
+
+    @Around("within(org.apache.nifi.web.NiFiServiceFacade+) && "
             + "execution(* get*(..))")
     public Object getLock(ProceedingJoinPoint proceedingJoinPoint) throws Throwable {
         return proceedWithReadLock(proceedingJoinPoint);


[7/9] nifi git commit: NIFI-4224: - Initial implementation of Process Group level Variable Registry - Updated to incorporate PR Feedback - Changed log message because slf4j-simple apparently has a memory leak; passing a String instead of passing in the C

Posted by mc...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/ExpressionCompiler.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/ExpressionCompiler.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/ExpressionCompiler.java
new file mode 100644
index 0000000..53a27fe
--- /dev/null
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/ExpressionCompiler.java
@@ -0,0 +1,1122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.attribute.expression.language.compile;
+
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ALL_ATTRIBUTES;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ALL_DELINEATED_VALUES;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ALL_MATCHING_ATTRIBUTES;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.AND;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ANY_ATTRIBUTE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ANY_DELINEATED_VALUE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ANY_MATCHING_ATTRIBUTE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.APPEND;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ATTRIBUTE_REFERENCE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ATTR_NAME;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.BASE64_DECODE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.BASE64_ENCODE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.CONTAINS;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.COUNT;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.DECIMAL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.DIVIDE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ENDS_WITH;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.EQUALS;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.EQUALS_IGNORE_CASE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ESCAPE_CSV;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ESCAPE_HTML3;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ESCAPE_HTML4;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ESCAPE_JSON;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ESCAPE_XML;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.EXPRESSION;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.FALSE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.FIND;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.FORMAT;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.FROM_RADIX;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.GET_DELIMITED_FIELD;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.GET_STATE_VALUE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.GREATER_THAN;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.GREATER_THAN_OR_EQUAL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.HOSTNAME;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IF_ELSE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IN;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.INDEX_OF;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IP;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IS_EMPTY;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IS_NULL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.JOIN;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.JSON_PATH;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LAST_INDEX_OF;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LENGTH;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LESS_THAN;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LESS_THAN_OR_EQUAL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MATCHES;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MATH;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MINUS;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MOD;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MULTIPLY;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MULTI_ATTRIBUTE_REFERENCE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NEXT_INT;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NOT;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NOT_NULL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NOW;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.OR;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.PLUS;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.PREPEND;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.RANDOM;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE_ALL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE_EMPTY;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE_FIRST;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE_NULL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.STARTS_WITH;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.STRING_LITERAL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_AFTER;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_AFTER_LAST;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_BEFORE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_BEFORE_LAST;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_DATE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_DECIMAL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_LITERAL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_LOWER;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_NUMBER;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_RADIX;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_STRING;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_UPPER;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TRIM;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TRUE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UNESCAPE_CSV;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UNESCAPE_HTML3;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UNESCAPE_HTML4;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UNESCAPE_JSON;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UNESCAPE_XML;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.URL_DECODE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.URL_ENCODE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UUID;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.WHOLE_NUMBER;
+
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.antlr.runtime.ANTLRStringStream;
+import org.antlr.runtime.CharStream;
+import org.antlr.runtime.CommonTokenStream;
+import org.antlr.runtime.tree.Tree;
+import org.apache.nifi.attribute.expression.language.Query;
+import org.apache.nifi.attribute.expression.language.Query.Range;
+import org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionLexer;
+import org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser;
+import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.DateEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.cast.BooleanCastEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.cast.DateCastEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.cast.DecimalCastEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.cast.NumberCastEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.cast.StringCastEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.cast.WholeNumberCastEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.AndEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.AppendEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.Base64DecodeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.Base64EncodeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.CharSequenceTranslatorEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.ContainsEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.DivideEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.EndsWithEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.EqualsEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.EqualsIgnoreCaseEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.FindEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.FormatEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.FromRadixEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.GetDelimitedFieldEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.GetStateVariableEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.GreaterThanEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.GreaterThanOrEqualEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.HostnameEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.IPEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.IfElseEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.InEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.IndexOfEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.IsEmptyEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.IsNullEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.JsonPathEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.LastIndexOfEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.LengthEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.LessThanEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.LessThanOrEqualEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.MatchesEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.MathEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.MinusEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.ModEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.MultiplyEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.NotEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.NotNullEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.NowEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.NumberToDateEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.OneUpSequenceEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.OrEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.PlusEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.PrependEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.RandomNumberGeneratorEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceAllEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceEmptyEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceFirstEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceNullEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.StartsWithEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.StringToDateEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringAfterEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringAfterLastEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringBeforeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringBeforeLastEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.ToLowerEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.ToRadixEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.ToStringEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.ToUpperEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.TrimEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.UrlDecodeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.UrlEncodeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.UuidEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.literals.BooleanLiteralEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.literals.DecimalLiteralEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.literals.StringLiteralEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.literals.ToLiteralEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.literals.WholeNumberLiteralEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.reduce.CountEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.reduce.JoinEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.reduce.ReduceEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.AllAttributesEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.AnyAttributeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.AttributeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.DelineatedAttributeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.IteratingEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.MappingEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiAttributeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiMatchAttributeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiNamedAttributeEvaluator;
+import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageException;
+import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException;
+import org.apache.nifi.expression.AttributeExpression.ResultType;
+import org.apache.nifi.flowfile.FlowFile;
+
+public class ExpressionCompiler {
+    private final Set<Evaluator<?>> evaluators = new HashSet<>();
+
+    public CompiledExpression compile(final String expression) {
+        try {
+            final CharStream input = new ANTLRStringStream(expression);
+            final AttributeExpressionLexer lexer = new AttributeExpressionLexer(input);
+            final CommonTokenStream lexerTokenStream = new CommonTokenStream(lexer);
+
+            final AttributeExpressionParser parser = new AttributeExpressionParser(lexerTokenStream);
+            final Tree ast = (Tree) parser.query().getTree();
+            final Tree tree = ast.getChild(0);
+
+            final Evaluator<?> evaluator = buildEvaluator(tree);
+            verifyMappingEvaluatorReduced(evaluator);
+
+            final Set<Evaluator<?>> allEvaluators = new HashSet<>(evaluators);
+            this.evaluators.clear();
+
+            return new CompiledExpression(expression, evaluator, tree, allEvaluators);
+        } catch (final AttributeExpressionLanguageParsingException e) {
+            throw e;
+        } catch (final Exception e) {
+            throw new AttributeExpressionLanguageParsingException(e);
+        }
+    }
+
+    private Tree compileTree(final String expression) throws AttributeExpressionLanguageParsingException {
+        try {
+            final CharStream input = new ANTLRStringStream(expression);
+            final AttributeExpressionLexer lexer = new AttributeExpressionLexer(input);
+            final CommonTokenStream lexerTokenStream = new CommonTokenStream(lexer);
+
+            final AttributeExpressionParser parser = new AttributeExpressionParser(lexerTokenStream);
+            final Tree ast = (Tree) parser.query().getTree();
+            final Tree tree = ast.getChild(0);
+
+            // ensure that we are able to build the evaluators, so that we validate syntax
+            final Evaluator<?> evaluator = buildEvaluator(tree);
+            verifyMappingEvaluatorReduced(evaluator);
+
+            return tree;
+        } catch (final AttributeExpressionLanguageParsingException e) {
+            throw e;
+        } catch (final Exception e) {
+            throw new AttributeExpressionLanguageParsingException(e);
+        }
+    }
+
+    private void verifyMappingEvaluatorReduced(final Evaluator<?> evaluator) {
+        final Evaluator<?> rightMostEvaluator;
+        if (evaluator instanceof IteratingEvaluator) {
+            rightMostEvaluator = ((IteratingEvaluator<?>) evaluator).getLogicEvaluator();
+        } else {
+            rightMostEvaluator = evaluator;
+        }
+
+        Evaluator<?> eval = rightMostEvaluator.getSubjectEvaluator();
+        Evaluator<?> lastEval = rightMostEvaluator;
+        while (eval != null) {
+            if (eval instanceof ReduceEvaluator) {
+                throw new AttributeExpressionLanguageParsingException("Expression attempts to call function '" + lastEval.getToken() + "' on the result of '" + eval.getToken() +
+                    "'. This is not allowed. Instead, use \"${literal( ${<embedded expression>} ):" + lastEval.getToken() + "(...)}\"");
+            }
+
+            lastEval = eval;
+            eval = eval.getSubjectEvaluator();
+        }
+
+        // if the result type of the evaluator is BOOLEAN, then it will always
+        // be reduced when evaluator.
+        final ResultType resultType = evaluator.getResultType();
+        if (resultType == ResultType.BOOLEAN) {
+            return;
+        }
+
+        final Evaluator<?> rootEvaluator = getRootSubjectEvaluator(evaluator);
+        if (rootEvaluator != null && rootEvaluator instanceof MultiAttributeEvaluator) {
+            final MultiAttributeEvaluator multiAttrEval = (MultiAttributeEvaluator) rootEvaluator;
+            switch (multiAttrEval.getEvaluationType()) {
+                case ALL_ATTRIBUTES:
+                case ALL_MATCHING_ATTRIBUTES:
+                case ALL_DELINEATED_VALUES: {
+                    if (!(evaluator instanceof ReduceEvaluator)) {
+                        throw new AttributeExpressionLanguageParsingException("Cannot evaluate expression because it attempts to reference multiple attributes but does not use a reducing function");
+                    }
+                    break;
+                }
+                default:
+                    throw new AttributeExpressionLanguageParsingException("Cannot evaluate expression because it attempts to reference multiple attributes but does not use a reducing function");
+            }
+        }
+    }
+
+    private Evaluator<?> getRootSubjectEvaluator(final Evaluator<?> evaluator) {
+        if (evaluator == null) {
+            return null;
+        }
+
+        final Evaluator<?> subject = evaluator.getSubjectEvaluator();
+        if (subject == null) {
+            return evaluator;
+        }
+
+        return getRootSubjectEvaluator(subject);
+    }
+
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    private Evaluator<?> buildExpressionEvaluator(final Tree tree) {
+        if (tree.getChildCount() == 0) {
+            throw new AttributeExpressionLanguageParsingException("EXPRESSION tree node has no children");
+        }
+
+        final Evaluator<?> evaluator;
+        if (tree.getChildCount() == 1) {
+            evaluator = buildEvaluator(tree.getChild(0));
+        } else {
+            // we can chain together functions in the form of:
+            // ${x:trim():substring(1,2):trim()}
+            // in this case, the subject of the right-most function is the function to its left; its
+            // subject is the function to its left (the first trim()), and its subject is the value of
+            // the 'x' attribute. We accomplish this logic by iterating over all of the children of the
+            // tree from the right-most child going left-ward.
+            evaluator = buildFunctionExpressionEvaluator(tree, 0);
+        }
+
+        Evaluator<?> chosenEvaluator = evaluator;
+        final Evaluator<?> rootEvaluator = getRootSubjectEvaluator(evaluator);
+        if (rootEvaluator != null) {
+            if (rootEvaluator instanceof MultiAttributeEvaluator) {
+                final MultiAttributeEvaluator multiAttrEval = (MultiAttributeEvaluator) rootEvaluator;
+
+                switch (multiAttrEval.getEvaluationType()) {
+                    case ANY_ATTRIBUTE:
+                    case ANY_MATCHING_ATTRIBUTE:
+                    case ANY_DELINEATED_VALUE:
+                        chosenEvaluator = new AnyAttributeEvaluator((BooleanEvaluator) evaluator, multiAttrEval);
+                        break;
+                    case ALL_ATTRIBUTES:
+                    case ALL_MATCHING_ATTRIBUTES:
+                    case ALL_DELINEATED_VALUES: {
+                        final ResultType resultType = evaluator.getResultType();
+                        if (resultType == ResultType.BOOLEAN) {
+                            chosenEvaluator = new AllAttributesEvaluator((BooleanEvaluator) evaluator, multiAttrEval);
+                        } else if (evaluator instanceof ReduceEvaluator) {
+                            chosenEvaluator = new MappingEvaluator((ReduceEvaluator) evaluator, multiAttrEval);
+                        } else {
+                            throw new AttributeExpressionLanguageException("Cannot evaluate Expression because it attempts to reference multiple attributes but does not use a reducing function");
+                        }
+                        break;
+                    }
+                }
+
+                evaluators.add(chosenEvaluator);
+                switch (multiAttrEval.getEvaluationType()) {
+                    case ANY_ATTRIBUTE:
+                        chosenEvaluator.setToken("anyAttribute");
+                        break;
+                    case ANY_MATCHING_ATTRIBUTE:
+                        chosenEvaluator.setToken("anyMatchingAttribute");
+                        break;
+                    case ANY_DELINEATED_VALUE:
+                        chosenEvaluator.setToken("anyDelineatedValue");
+                        break;
+                    case ALL_ATTRIBUTES:
+                        chosenEvaluator.setToken("allAttributes");
+                        break;
+                    case ALL_MATCHING_ATTRIBUTES:
+                        chosenEvaluator.setToken("allMatchingAttributes");
+                        break;
+                    case ALL_DELINEATED_VALUES:
+                        chosenEvaluator.setToken("allDelineatedValues");
+                        break;
+                }
+            }
+        }
+
+        return chosenEvaluator;
+    }
+
+    private Evaluator<?> buildFunctionExpressionEvaluator(final Tree tree, final int offset) {
+        if (tree.getChildCount() == 0) {
+            throw new AttributeExpressionLanguageParsingException("EXPRESSION tree node has no children");
+        }
+        final int firstChildIndex = tree.getChildCount() - offset - 1;
+        if (firstChildIndex == 0) {
+            return buildEvaluator(tree.getChild(0));
+        }
+
+        final Tree functionTree = tree.getChild(firstChildIndex);
+        final Evaluator<?> subjectEvaluator = buildFunctionExpressionEvaluator(tree, offset + 1);
+
+        final Tree functionNameTree = functionTree.getChild(0);
+        final List<Evaluator<?>> argEvaluators = new ArrayList<>();
+        for (int i = 1; i < functionTree.getChildCount(); i++) {
+            argEvaluators.add(buildEvaluator(functionTree.getChild(i)));
+        }
+        return buildFunctionEvaluator(functionNameTree, subjectEvaluator, argEvaluators);
+    }
+
+    private List<Evaluator<?>> verifyArgCount(final List<Evaluator<?>> args, final int count, final String functionName) {
+        if (args.size() != count) {
+            throw new AttributeExpressionLanguageParsingException(functionName + "() function takes " + count + " arguments");
+        }
+        return args;
+    }
+
+    private Evaluator<String> toStringEvaluator(final Evaluator<?> evaluator) {
+        return toStringEvaluator(evaluator, null);
+    }
+
+    private Evaluator<String> toStringEvaluator(final Evaluator<?> evaluator, final String location) {
+        if (evaluator.getResultType() == ResultType.STRING) {
+            return (StringEvaluator) evaluator;
+        }
+
+        return addToken(new StringCastEvaluator(evaluator), evaluator.getToken());
+    }
+
+    @SuppressWarnings("unchecked")
+    private Evaluator<Boolean> toBooleanEvaluator(final Evaluator<?> evaluator, final String location) {
+        switch (evaluator.getResultType()) {
+            case BOOLEAN:
+                return (Evaluator<Boolean>) evaluator;
+            case STRING:
+                return addToken(new BooleanCastEvaluator((StringEvaluator) evaluator), evaluator.getToken());
+            default:
+                throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + evaluator.getResultType() + " to " + ResultType.BOOLEAN
+                    + (location == null ? "" : " at location [" + location + "]"));
+        }
+
+    }
+
+    private Evaluator<Boolean> toBooleanEvaluator(final Evaluator<?> evaluator) {
+        return toBooleanEvaluator(evaluator, null);
+    }
+
+    private Evaluator<Long> toWholeNumberEvaluator(final Evaluator<?> evaluator) {
+        return toWholeNumberEvaluator(evaluator, null);
+    }
+
+    @SuppressWarnings("unchecked")
+    private Evaluator<Long> toWholeNumberEvaluator(final Evaluator<?> evaluator, final String location) {
+        switch (evaluator.getResultType()) {
+            case WHOLE_NUMBER:
+                return (Evaluator<Long>) evaluator;
+            case STRING:
+            case DATE:
+            case DECIMAL:
+            case NUMBER:
+                return addToken(new WholeNumberCastEvaluator(evaluator), evaluator.getToken());
+            default:
+                throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + evaluator.getResultType() + " to " + ResultType.WHOLE_NUMBER
+                    + (location == null ? "" : " at location [" + location + "]"));
+        }
+    }
+
+    private Evaluator<Double> toDecimalEvaluator(final Evaluator<?> evaluator) {
+        return toDecimalEvaluator(evaluator, null);
+    }
+
+    @SuppressWarnings("unchecked")
+    private Evaluator<Double> toDecimalEvaluator(final Evaluator<?> evaluator, final String location) {
+        switch (evaluator.getResultType()) {
+            case DECIMAL:
+                return (Evaluator<Double>) evaluator;
+            case WHOLE_NUMBER:
+            case STRING:
+            case DATE:
+            case NUMBER:
+                return addToken(new DecimalCastEvaluator(evaluator), evaluator.getToken());
+            default:
+                throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + evaluator.getResultType() + " to " + ResultType.DECIMAL
+                    + (location == null ? "" : " at location [" + location + "]"));
+        }
+    }
+
+    private Evaluator<Number> toNumberEvaluator(final Evaluator<?> evaluator) {
+        return toNumberEvaluator(evaluator, null);
+    }
+
+    @SuppressWarnings("unchecked")
+    private Evaluator<Number> toNumberEvaluator(final Evaluator<?> evaluator, final String location) {
+        switch (evaluator.getResultType()) {
+            case NUMBER:
+                return (Evaluator<Number>) evaluator;
+            case STRING:
+            case DATE:
+            case DECIMAL:
+            case WHOLE_NUMBER:
+                return addToken(new NumberCastEvaluator(evaluator), evaluator.getToken());
+            default:
+                throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + evaluator.getResultType() + " to " + ResultType.WHOLE_NUMBER
+                    + (location == null ? "" : " at location [" + location + "]"));
+        }
+    }
+
+    private DateEvaluator toDateEvaluator(final Evaluator<?> evaluator) {
+        return toDateEvaluator(evaluator, null);
+    }
+
+    private DateEvaluator toDateEvaluator(final Evaluator<?> evaluator, final String location) {
+        if (evaluator.getResultType() == ResultType.DATE) {
+            return (DateEvaluator) evaluator;
+        }
+
+        return new DateCastEvaluator(evaluator);
+    }
+
+    private Evaluator<?> buildFunctionEvaluator(final Tree tree, final Evaluator<?> subjectEvaluator, final List<Evaluator<?>> argEvaluators) {
+        switch (tree.getType()) {
+            case TRIM: {
+                verifyArgCount(argEvaluators, 0, "trim");
+                return addToken(new TrimEvaluator(toStringEvaluator(subjectEvaluator)), "trim");
+            }
+            case TO_STRING: {
+                verifyArgCount(argEvaluators, 0, "toString");
+                return addToken(new ToStringEvaluator(subjectEvaluator), "toString");
+            }
+            case TO_LOWER: {
+                verifyArgCount(argEvaluators, 0, "toLower");
+                return addToken(new ToLowerEvaluator(toStringEvaluator(subjectEvaluator)), "toLower");
+            }
+            case TO_UPPER: {
+                verifyArgCount(argEvaluators, 0, "toUpper");
+                return addToken(new ToUpperEvaluator(toStringEvaluator(subjectEvaluator)), "toUpper");
+            }
+            case URL_ENCODE: {
+                verifyArgCount(argEvaluators, 0, "urlEncode");
+                return addToken(new UrlEncodeEvaluator(toStringEvaluator(subjectEvaluator)), "urlEncode");
+            }
+            case URL_DECODE: {
+                verifyArgCount(argEvaluators, 0, "urlDecode");
+                return addToken(new UrlDecodeEvaluator(toStringEvaluator(subjectEvaluator)), "urlDecode");
+            }
+            case BASE64_ENCODE: {
+                verifyArgCount(argEvaluators, 0, "base64Encode");
+                return addToken(new Base64EncodeEvaluator(toStringEvaluator(subjectEvaluator)), "base64Encode");
+            }
+            case BASE64_DECODE: {
+                verifyArgCount(argEvaluators, 0, "base64Decode");
+                return addToken(new Base64DecodeEvaluator(toStringEvaluator(subjectEvaluator)), "base64Decode");
+            }
+            case ESCAPE_CSV: {
+                verifyArgCount(argEvaluators, 0, "escapeCsv");
+                return addToken(CharSequenceTranslatorEvaluator.csvEscapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
+            }
+            case ESCAPE_HTML3: {
+                verifyArgCount(argEvaluators, 0, "escapeHtml3");
+                return addToken(CharSequenceTranslatorEvaluator.html3EscapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
+            }
+            case ESCAPE_HTML4: {
+                verifyArgCount(argEvaluators, 0, "escapeHtml4");
+                return addToken(CharSequenceTranslatorEvaluator.html4EscapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
+            }
+            case ESCAPE_JSON: {
+                verifyArgCount(argEvaluators, 0, "escapeJson");
+                return addToken(CharSequenceTranslatorEvaluator.jsonEscapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
+            }
+            case ESCAPE_XML: {
+                verifyArgCount(argEvaluators, 0, "escapeXml");
+                return addToken(CharSequenceTranslatorEvaluator.xmlEscapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
+            }
+            case UNESCAPE_CSV: {
+                verifyArgCount(argEvaluators, 0, "unescapeCsv");
+                return addToken(CharSequenceTranslatorEvaluator.csvUnescapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
+            }
+            case UNESCAPE_HTML3: {
+                verifyArgCount(argEvaluators, 0, "unescapeHtml3");
+                return addToken(CharSequenceTranslatorEvaluator.html3UnescapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
+            }
+            case UNESCAPE_HTML4: {
+                verifyArgCount(argEvaluators, 0, "unescapeHtml4");
+                return addToken(CharSequenceTranslatorEvaluator.html4UnescapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
+            }
+            case UNESCAPE_JSON: {
+                verifyArgCount(argEvaluators, 0, "unescapeJson");
+                return addToken(CharSequenceTranslatorEvaluator.jsonUnescapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
+            }
+            case UNESCAPE_XML: {
+                verifyArgCount(argEvaluators, 0, "unescapeXml");
+                return addToken(CharSequenceTranslatorEvaluator.xmlUnescapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
+            }
+            case SUBSTRING_BEFORE: {
+                verifyArgCount(argEvaluators, 1, "substringBefore");
+                return addToken(new SubstringBeforeEvaluator(toStringEvaluator(subjectEvaluator),
+                    toStringEvaluator(argEvaluators.get(0), "first argument to substringBefore")), "substringBefore");
+            }
+            case SUBSTRING_BEFORE_LAST: {
+                verifyArgCount(argEvaluators, 1, "substringBeforeLast");
+                return addToken(new SubstringBeforeLastEvaluator(toStringEvaluator(subjectEvaluator),
+                    toStringEvaluator(argEvaluators.get(0), "first argument to substringBeforeLast")), "substringBeforeLast");
+            }
+            case SUBSTRING_AFTER: {
+                verifyArgCount(argEvaluators, 1, "substringAfter");
+                return addToken(new SubstringAfterEvaluator(toStringEvaluator(subjectEvaluator),
+                    toStringEvaluator(argEvaluators.get(0), "first argument to substringAfter")), "substringAfter");
+            }
+            case SUBSTRING_AFTER_LAST: {
+                verifyArgCount(argEvaluators, 1, "substringAfterLast");
+                return addToken(new SubstringAfterLastEvaluator(toStringEvaluator(subjectEvaluator),
+                    toStringEvaluator(argEvaluators.get(0), "first argument to substringAfterLast")), "substringAfterLast");
+            }
+            case REPLACE_NULL: {
+                verifyArgCount(argEvaluators, 1, "replaceNull");
+                return addToken(new ReplaceNullEvaluator(toStringEvaluator(subjectEvaluator),
+                    toStringEvaluator(argEvaluators.get(0), "first argument to replaceNull")), "replaceNull");
+            }
+            case REPLACE_EMPTY: {
+                verifyArgCount(argEvaluators, 1, "replaceEmtpy");
+                return addToken(new ReplaceEmptyEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0), "first argument to replaceEmpty")), "replaceEmpty");
+            }
+            case REPLACE: {
+                verifyArgCount(argEvaluators, 2, "replace");
+                return addToken(new ReplaceEvaluator(toStringEvaluator(subjectEvaluator),
+                    toStringEvaluator(argEvaluators.get(0), "first argument to replace"),
+                    toStringEvaluator(argEvaluators.get(1), "second argument to replace")), "replace");
+            }
+            case REPLACE_FIRST: {
+                verifyArgCount(argEvaluators, 2, "replaceFirst");
+                return addToken(new ReplaceFirstEvaluator(toStringEvaluator(subjectEvaluator),
+                    toStringEvaluator(argEvaluators.get(0), "first argument to replaceFirst"),
+                    toStringEvaluator(argEvaluators.get(1), "second argument to replaceFirst")), "replaceFirst");
+            }
+            case REPLACE_ALL: {
+                verifyArgCount(argEvaluators, 2, "replaceAll");
+                return addToken(new ReplaceAllEvaluator(toStringEvaluator(subjectEvaluator),
+                    toStringEvaluator(argEvaluators.get(0), "first argument to replaceAll"),
+                    toStringEvaluator(argEvaluators.get(1), "second argument to replaceAll")), "replaceAll");
+            }
+            case APPEND: {
+                verifyArgCount(argEvaluators, 1, "append");
+                return addToken(new AppendEvaluator(toStringEvaluator(subjectEvaluator),
+                    toStringEvaluator(argEvaluators.get(0), "first argument to append")), "append");
+            }
+            case PREPEND: {
+                verifyArgCount(argEvaluators, 1, "prepend");
+                return addToken(new PrependEvaluator(toStringEvaluator(subjectEvaluator),
+                    toStringEvaluator(argEvaluators.get(0), "first argument to prepend")), "prepend");
+            }
+            case SUBSTRING: {
+                final int numArgs = argEvaluators.size();
+                if (numArgs == 1) {
+                    return addToken(new SubstringEvaluator(toStringEvaluator(subjectEvaluator),
+                        toWholeNumberEvaluator(argEvaluators.get(0), "first argument to substring")), "substring");
+                } else if (numArgs == 2) {
+                    return addToken(new SubstringEvaluator(toStringEvaluator(subjectEvaluator),
+                        toWholeNumberEvaluator(argEvaluators.get(0), "first argument to substring"),
+                        toWholeNumberEvaluator(argEvaluators.get(1), "second argument to substring")), "substring");
+                } else {
+                    throw new AttributeExpressionLanguageParsingException("substring() function can take either 1 or 2 arguments but cannot take " + numArgs + " arguments");
+                }
+            }
+            case JOIN: {
+                verifyArgCount(argEvaluators, 1, "join");
+                return addToken(new JoinEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0))), "join");
+            }
+            case COUNT: {
+                verifyArgCount(argEvaluators, 0, "count");
+                return addToken(new CountEvaluator(subjectEvaluator), "count");
+            }
+            case IS_NULL: {
+                verifyArgCount(argEvaluators, 0, "isNull");
+                return addToken(new IsNullEvaluator(toStringEvaluator(subjectEvaluator)), "isNull");
+            }
+            case IS_EMPTY: {
+                verifyArgCount(argEvaluators, 0, "isEmpty");
+                return addToken(new IsEmptyEvaluator(toStringEvaluator(subjectEvaluator)), "isEmpty");
+            }
+            case NOT_NULL: {
+                verifyArgCount(argEvaluators, 0, "notNull");
+                return addToken(new NotNullEvaluator(toStringEvaluator(subjectEvaluator)), "notNull");
+            }
+            case STARTS_WITH: {
+                verifyArgCount(argEvaluators, 1, "startsWith");
+                return addToken(new StartsWithEvaluator(toStringEvaluator(subjectEvaluator),
+                    toStringEvaluator(argEvaluators.get(0), "first argument to startsWith")), "startsWith");
+            }
+            case ENDS_WITH: {
+                verifyArgCount(argEvaluators, 1, "endsWith");
+                return addToken(new EndsWithEvaluator(toStringEvaluator(subjectEvaluator),
+                    toStringEvaluator(argEvaluators.get(0), "first argument to endsWith")), "endsWith");
+            }
+            case CONTAINS: {
+                verifyArgCount(argEvaluators, 1, "contains");
+                return addToken(new ContainsEvaluator(toStringEvaluator(subjectEvaluator),
+                    toStringEvaluator(argEvaluators.get(0), "first argument to contains")), "contains");
+            }
+            case IN: {
+                List<Evaluator<String>> list = new ArrayList<Evaluator<String>>();
+                for (int i = 0; i < argEvaluators.size(); i++) {
+                    list.add(toStringEvaluator(argEvaluators.get(i), i + "th argument to in"));
+                }
+                return addToken(new InEvaluator(toStringEvaluator(subjectEvaluator), list), "in");
+            }
+            case FIND: {
+                verifyArgCount(argEvaluators, 1, "find");
+                return addToken(new FindEvaluator(toStringEvaluator(subjectEvaluator),
+                    toStringEvaluator(argEvaluators.get(0), "first argument to find")), "find");
+            }
+            case MATCHES: {
+                verifyArgCount(argEvaluators, 1, "matches");
+                return addToken(new MatchesEvaluator(toStringEvaluator(subjectEvaluator),
+                    toStringEvaluator(argEvaluators.get(0), "first argument to matches")), "matches");
+            }
+            case EQUALS: {
+                verifyArgCount(argEvaluators, 1, "equals");
+                return addToken(new EqualsEvaluator(subjectEvaluator, argEvaluators.get(0)), "equals");
+            }
+            case EQUALS_IGNORE_CASE: {
+                verifyArgCount(argEvaluators, 1, "equalsIgnoreCase");
+                return addToken(new EqualsIgnoreCaseEvaluator(toStringEvaluator(subjectEvaluator),
+                    toStringEvaluator(argEvaluators.get(0), "first argument to equalsIgnoreCase")), "equalsIgnoreCase");
+            }
+            case GREATER_THAN: {
+                verifyArgCount(argEvaluators, 1, "gt");
+                return addToken(new GreaterThanEvaluator(toNumberEvaluator(subjectEvaluator),
+                    toNumberEvaluator(argEvaluators.get(0), "first argument to gt")), "gt");
+            }
+            case GREATER_THAN_OR_EQUAL: {
+                verifyArgCount(argEvaluators, 1, "ge");
+                return addToken(new GreaterThanOrEqualEvaluator(toNumberEvaluator(subjectEvaluator),
+                    toNumberEvaluator(argEvaluators.get(0), "first argument to ge")), "ge");
+            }
+            case LESS_THAN: {
+                verifyArgCount(argEvaluators, 1, "lt");
+                return addToken(new LessThanEvaluator(toNumberEvaluator(subjectEvaluator),
+                    toNumberEvaluator(argEvaluators.get(0), "first argument to lt")), "lt");
+            }
+            case LESS_THAN_OR_EQUAL: {
+                verifyArgCount(argEvaluators, 1, "le");
+                return addToken(new LessThanOrEqualEvaluator(toNumberEvaluator(subjectEvaluator),
+                    toNumberEvaluator(argEvaluators.get(0), "first argument to le")), "le");
+            }
+            case LENGTH: {
+                verifyArgCount(argEvaluators, 0, "length");
+                return addToken(new LengthEvaluator(toStringEvaluator(subjectEvaluator)), "length");
+            }
+            case TO_DATE: {
+                if (argEvaluators.isEmpty()) {
+                    return addToken(new NumberToDateEvaluator(toWholeNumberEvaluator(subjectEvaluator)), "toDate");
+                } else if (subjectEvaluator.getResultType() == ResultType.STRING && argEvaluators.size() == 1) {
+                    return addToken(new StringToDateEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0)), null), "toDate");
+                } else if (subjectEvaluator.getResultType() == ResultType.STRING && argEvaluators.size() == 2) {
+                    return addToken(new StringToDateEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0)), toStringEvaluator(argEvaluators.get(1))), "toDate");
+                } else {
+                    return addToken(new NumberToDateEvaluator(toWholeNumberEvaluator(subjectEvaluator)), "toDate");
+                }
+            }
+            case TO_NUMBER: {
+                verifyArgCount(argEvaluators, 0, "toNumber");
+                switch (subjectEvaluator.getResultType()) {
+                    case STRING:
+                    case WHOLE_NUMBER:
+                    case DECIMAL:
+                    case NUMBER:
+                    case DATE:
+                        return addToken(toWholeNumberEvaluator(subjectEvaluator), "toNumber");
+                    default:
+                        throw new AttributeExpressionLanguageParsingException(subjectEvaluator + " returns type " + subjectEvaluator.getResultType() + " but expected to get " + ResultType.STRING +
+                            ", " + ResultType.DECIMAL + ", or " + ResultType.DATE);
+                }
+            }
+            case TO_DECIMAL: {
+                verifyArgCount(argEvaluators, 0, "toDecimal");
+                switch (subjectEvaluator.getResultType()) {
+                    case WHOLE_NUMBER:
+                    case DECIMAL:
+                    case STRING:
+                    case NUMBER:
+                    case DATE:
+                        return addToken(toDecimalEvaluator(subjectEvaluator), "toDecimal");
+                    default:
+                        throw new AttributeExpressionLanguageParsingException(subjectEvaluator + " returns type " + subjectEvaluator.getResultType() + " but expected to get " + ResultType.STRING +
+                            ", " + ResultType.WHOLE_NUMBER + ", or " + ResultType.DATE);
+                }
+            }
+            case TO_RADIX: {
+                if (argEvaluators.size() == 1) {
+                    return addToken(new ToRadixEvaluator(toWholeNumberEvaluator(subjectEvaluator),
+                        toWholeNumberEvaluator(argEvaluators.get(0))), "toRadix");
+                } else {
+                    return addToken(new ToRadixEvaluator(toWholeNumberEvaluator(subjectEvaluator),
+                        toWholeNumberEvaluator(argEvaluators.get(0)), toWholeNumberEvaluator(argEvaluators.get(1))), "toRadix");
+                }
+            }
+            case FROM_RADIX: {
+                return addToken(new FromRadixEvaluator(toStringEvaluator(subjectEvaluator),
+                    toWholeNumberEvaluator(argEvaluators.get(0))), "fromRadix");
+            }
+            case MOD: {
+                return addToken(new ModEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "mod");
+            }
+            case PLUS: {
+                return addToken(new PlusEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "plus");
+            }
+            case MINUS: {
+                return addToken(new MinusEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "minus");
+            }
+            case MULTIPLY: {
+                return addToken(new MultiplyEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "multiply");
+            }
+            case DIVIDE: {
+                return addToken(new DivideEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "divide");
+            }
+            case MATH: {
+                if (argEvaluators.size() == 1) {
+                    return addToken(new MathEvaluator(toNumberEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0)), null), "math");
+                } else if (argEvaluators.size() == 2) {
+                    return addToken(new MathEvaluator(toNumberEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0)), toNumberEvaluator(argEvaluators.get(1))), "math");
+                } else {
+                    throw new AttributeExpressionLanguageParsingException("math() function takes 1 or 2 arguments");
+                }
+            }
+            case RANDOM: {
+                return addToken(new RandomNumberGeneratorEvaluator(), "random");
+            }
+            case INDEX_OF: {
+                verifyArgCount(argEvaluators, 1, "indexOf");
+                return addToken(new IndexOfEvaluator(toStringEvaluator(subjectEvaluator),
+                    toStringEvaluator(argEvaluators.get(0), "first argument to indexOf")), "indexOf");
+            }
+            case LAST_INDEX_OF: {
+                verifyArgCount(argEvaluators, 1, "lastIndexOf");
+                return addToken(new LastIndexOfEvaluator(toStringEvaluator(subjectEvaluator),
+                    toStringEvaluator(argEvaluators.get(0), "first argument to lastIndexOf")), "lastIndexOf");
+            }
+            case FORMAT: {
+                if (argEvaluators.size() == 1) {
+                    return addToken(new FormatEvaluator(toDateEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0), "first argument of format"), null), "format");
+                } else if (argEvaluators.size() == 2) {
+                    return addToken(new FormatEvaluator(toDateEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0)), toStringEvaluator(argEvaluators.get(1))), "format");
+                } else {
+                    throw new AttributeExpressionLanguageParsingException("format() function takes 1 or 2 arguments");
+                }
+            }
+            case OR: {
+                return addToken(new OrEvaluator(toBooleanEvaluator(subjectEvaluator), toBooleanEvaluator(argEvaluators.get(0))), "or");
+            }
+            case AND: {
+                return addToken(new AndEvaluator(toBooleanEvaluator(subjectEvaluator), toBooleanEvaluator(argEvaluators.get(0))), "and");
+            }
+            case NOT: {
+                return addToken(new NotEvaluator(toBooleanEvaluator(subjectEvaluator)), "not");
+            }
+            case GET_DELIMITED_FIELD: {
+                if (argEvaluators.size() == 1) {
+                    // Only a single argument - the index to return.
+                    return addToken(new GetDelimitedFieldEvaluator(toStringEvaluator(subjectEvaluator),
+                        toWholeNumberEvaluator(argEvaluators.get(0), "first argument of getDelimitedField")), "getDelimitedField");
+                } else if (argEvaluators.size() == 2) {
+                    // two arguments - index and delimiter.
+                    return addToken(new GetDelimitedFieldEvaluator(toStringEvaluator(subjectEvaluator),
+                        toWholeNumberEvaluator(argEvaluators.get(0), "first argument of getDelimitedField"),
+                        toStringEvaluator(argEvaluators.get(1), "second argument of getDelimitedField")),
+                        "getDelimitedField");
+                } else if (argEvaluators.size() == 3) {
+                    // 3 arguments - index, delimiter, quote char.
+                    return addToken(new GetDelimitedFieldEvaluator(toStringEvaluator(subjectEvaluator),
+                        toWholeNumberEvaluator(argEvaluators.get(0), "first argument of getDelimitedField"),
+                        toStringEvaluator(argEvaluators.get(1), "second argument of getDelimitedField"),
+                        toStringEvaluator(argEvaluators.get(2), "third argument of getDelimitedField")),
+                        "getDelimitedField");
+                } else if (argEvaluators.size() == 4) {
+                    // 4 arguments - index, delimiter, quote char, escape char
+                    return addToken(new GetDelimitedFieldEvaluator(toStringEvaluator(subjectEvaluator),
+                        toWholeNumberEvaluator(argEvaluators.get(0), "first argument of getDelimitedField"),
+                        toStringEvaluator(argEvaluators.get(1), "second argument of getDelimitedField"),
+                        toStringEvaluator(argEvaluators.get(2), "third argument of getDelimitedField"),
+                        toStringEvaluator(argEvaluators.get(3), "fourth argument of getDelimitedField")),
+                        "getDelimitedField");
+                } else {
+                    // 5 arguments - index, delimiter, quote char, escape char, strip escape/quote chars flag
+                    return addToken(new GetDelimitedFieldEvaluator(toStringEvaluator(subjectEvaluator),
+                        toWholeNumberEvaluator(argEvaluators.get(0), "first argument of getDelimitedField"),
+                        toStringEvaluator(argEvaluators.get(1), "second argument of getDelimitedField"),
+                        toStringEvaluator(argEvaluators.get(2), "third argument of getDelimitedField"),
+                        toStringEvaluator(argEvaluators.get(3), "fourth argument of getDelimitedField"),
+                        toBooleanEvaluator(argEvaluators.get(4), "fifth argument of getDelimitedField")),
+                        "getDelimitedField");
+                }
+            }
+            case JSON_PATH: {
+                verifyArgCount(argEvaluators, 1, "jsonPath");
+                return addToken(new JsonPathEvaluator(toStringEvaluator(subjectEvaluator),
+                    toStringEvaluator(argEvaluators.get(0), "first argument to jsonPath")), "jsonPath");
+            }
+            case IF_ELSE: {
+                verifyArgCount(argEvaluators, 2, "ifElse");
+                return addToken(new IfElseEvaluator(toBooleanEvaluator(subjectEvaluator),
+                    toStringEvaluator(argEvaluators.get(0), "argument to return if true"),
+                    toStringEvaluator(argEvaluators.get(1), "argument to return if false")), "ifElse");
+            }
+            default:
+                throw new AttributeExpressionLanguageParsingException("Expected a Function-type expression but got " + tree.toString());
+        }
+    }
+
+    public Evaluator<?> buildEvaluator(final Tree tree) {
+        switch (tree.getType()) {
+            case EXPRESSION: {
+                return buildExpressionEvaluator(tree);
+            }
+            case ATTRIBUTE_REFERENCE: {
+                final Evaluator<?> childEvaluator = buildEvaluator(tree.getChild(0));
+                if (childEvaluator instanceof MultiAttributeEvaluator) {
+                    return childEvaluator;
+                }
+                final AttributeEvaluator eval = new AttributeEvaluator(toStringEvaluator(childEvaluator));
+                evaluators.add(eval);
+                return eval;
+            }
+            case MULTI_ATTRIBUTE_REFERENCE: {
+
+                final Tree functionTypeTree = tree.getChild(0);
+                final int multiAttrType = functionTypeTree.getType();
+                if (multiAttrType == ANY_DELINEATED_VALUE || multiAttrType == ALL_DELINEATED_VALUES) {
+                    final Evaluator<String> delineatedValueEvaluator = toStringEvaluator(buildEvaluator(tree.getChild(1)));
+                    final Evaluator<String> delimiterEvaluator = toStringEvaluator(buildEvaluator(tree.getChild(2)));
+
+                    final String token = (multiAttrType == ANY_DELINEATED_VALUE) ? "anyDelineatedValue" : "allDelineatedValues";
+                    return addToken(new DelineatedAttributeEvaluator(delineatedValueEvaluator, delimiterEvaluator, multiAttrType), token);
+                }
+
+                final List<String> attributeNames = new ArrayList<>();
+                for (int i = 1; i < tree.getChildCount(); i++) {  // skip the first child because that's the name of the multi-attribute function
+                    attributeNames.add(newStringLiteralEvaluator(tree.getChild(i).getText()).evaluate(null).getValue());
+                }
+
+                switch (multiAttrType) {
+                    case ALL_ATTRIBUTES:
+                        for (final String attributeName : attributeNames) {
+                            try {
+                                FlowFile.KeyValidator.validateKey(attributeName);
+                            } catch (final IllegalArgumentException iae) {
+                                throw new AttributeExpressionLanguageParsingException("Invalid Attribute Name: " + attributeName + ". " + iae.getMessage());
+                            }
+                        }
+
+                        return addToken(new MultiNamedAttributeEvaluator(attributeNames, ALL_ATTRIBUTES), "allAttributes");
+                    case ALL_MATCHING_ATTRIBUTES:
+                        return addToken(new MultiMatchAttributeEvaluator(attributeNames, ALL_MATCHING_ATTRIBUTES), "allMatchingAttributes");
+                    case ANY_ATTRIBUTE:
+                        for (final String attributeName : attributeNames) {
+                            try {
+                                FlowFile.KeyValidator.validateKey(attributeName);
+                            } catch (final IllegalArgumentException iae) {
+                                throw new AttributeExpressionLanguageParsingException("Invalid Attribute Name: " + attributeName + ". " + iae.getMessage());
+                            }
+                        }
+
+                        return addToken(new MultiNamedAttributeEvaluator(attributeNames, ANY_ATTRIBUTE), "anyAttribute");
+                    case ANY_MATCHING_ATTRIBUTE:
+                        return addToken(new MultiMatchAttributeEvaluator(attributeNames, ANY_MATCHING_ATTRIBUTE), "anyMatchingAttribute");
+                    default:
+                        throw new AssertionError("Illegal Multi-Attribute Reference: " + functionTypeTree.toString());
+                }
+            }
+            case ATTR_NAME: {
+                return newStringLiteralEvaluator(tree.getChild(0).getText());
+            }
+            case WHOLE_NUMBER: {
+                return addToken(new WholeNumberLiteralEvaluator(tree.getText()), "wholeNumber");
+            }
+            case STRING_LITERAL: {
+                return newStringLiteralEvaluator(tree.getText());
+            }
+            case DECIMAL: {
+                return addToken(new DecimalLiteralEvaluator(tree.getText()), "decimal");
+            }
+            case TRUE:
+            case FALSE:
+                return buildBooleanEvaluator(tree);
+            case UUID: {
+                return addToken(new UuidEvaluator(), "uuid");
+            }
+            case NOW: {
+                return addToken(new NowEvaluator(), "now");
+            }
+            case TO_LITERAL: {
+                final Evaluator<?> argEvaluator = buildEvaluator(tree.getChild(0));
+                return addToken(new ToLiteralEvaluator(argEvaluator), "toLiteral");
+            }
+            case IP: {
+                try {
+                    return addToken(new IPEvaluator(), "ip");
+                } catch (final UnknownHostException e) {
+                    throw new AttributeExpressionLanguageException(e);
+                }
+            }
+            case HOSTNAME: {
+                if (tree.getChildCount() == 0) {
+                    try {
+                        return addToken(new HostnameEvaluator(false), "hostname");
+                    } catch (final UnknownHostException e) {
+                        throw new AttributeExpressionLanguageException(e);
+                    }
+                } else if (tree.getChildCount() == 1) {
+                    final Tree childTree = tree.getChild(0);
+                    try {
+                        switch (childTree.getType()) {
+                            case TRUE:
+                                return addToken(new HostnameEvaluator(true), "hostname");
+                            case FALSE:
+                                return addToken(new HostnameEvaluator(false), "hostname");
+                            default:
+                                throw new AttributeExpressionLanguageParsingException("Call to hostname() must take 0 or 1 (boolean) parameter");
+                        }
+                    } catch (final UnknownHostException e) {
+                        throw new AttributeExpressionLanguageException(e);
+                    }
+                } else {
+                    throw new AttributeExpressionLanguageParsingException("Call to hostname() must take 0 or 1 (boolean) parameter");
+                }
+            }
+            case NEXT_INT: {
+                return addToken(new OneUpSequenceEvaluator(), "nextInt");
+            }
+            case RANDOM: {
+                return addToken(new RandomNumberGeneratorEvaluator(), "random");
+            }
+            case MATH: {
+                if (tree.getChildCount() == 1) {
+                    return addToken(new MathEvaluator(null, toStringEvaluator(buildEvaluator(tree.getChild(0))), null), "math");
+                } else {
+                    throw new AttributeExpressionLanguageParsingException("Call to math() as the subject must take exactly 1 parameter");
+                }
+            }
+            case GET_STATE_VALUE: {
+                final Tree childTree = tree.getChild(0);
+                final Evaluator<?> argEvaluator = buildEvaluator(childTree);
+                final Evaluator<String> stringEvaluator = toStringEvaluator(argEvaluator);
+                final GetStateVariableEvaluator eval = new GetStateVariableEvaluator(stringEvaluator);
+                evaluators.add(eval);
+                return eval;
+            }
+            default:
+                throw new AttributeExpressionLanguageParsingException("Unexpected token: " + tree.toString());
+        }
+    }
+
+    private <T> Evaluator<T> addToken(final Evaluator<T> evaluator, final String token) {
+        evaluator.setToken(token);
+        evaluators.add(evaluator);
+        return evaluator;
+    }
+
+
+    private Evaluator<String> newStringLiteralEvaluator(final String literalValue) {
+        if (literalValue == null || literalValue.length() < 2) {
+            return addToken(new StringLiteralEvaluator(literalValue), literalValue);
+        }
+
+        final List<Range> ranges = Query.extractExpressionRanges(literalValue);
+        if (ranges.isEmpty()) {
+            return addToken(new StringLiteralEvaluator(literalValue), literalValue);
+        }
+
+        final List<Evaluator<?>> evaluators = new ArrayList<>();
+
+        int lastIndex = 0;
+        for (final Range range : ranges) {
+            if (range.getStart() > lastIndex) {
+                evaluators.add(newStringLiteralEvaluator(literalValue.substring(lastIndex, range.getStart())));
+            }
+
+            final String treeText = literalValue.substring(range.getStart(), range.getEnd() + 1);
+            evaluators.add(buildEvaluator(compileTree(treeText)));
+            lastIndex = range.getEnd() + 1;
+        }
+
+        final Range lastRange = ranges.get(ranges.size() - 1);
+        if (lastRange.getEnd() + 1 < literalValue.length()) {
+            final String treeText = literalValue.substring(lastRange.getEnd() + 1);
+            evaluators.add(newStringLiteralEvaluator(treeText));
+        }
+
+        if (evaluators.size() == 1) {
+            return toStringEvaluator(evaluators.get(0));
+        }
+
+        Evaluator<String> lastEvaluator = toStringEvaluator(evaluators.get(0));
+        for (int i = 1; i < evaluators.size(); i++) {
+            lastEvaluator = new AppendEvaluator(lastEvaluator, toStringEvaluator(evaluators.get(i)));
+        }
+
+        this.evaluators.addAll(evaluators);
+        return lastEvaluator;
+    }
+
+
+    private Evaluator<Boolean> buildBooleanEvaluator(final Tree tree) {
+        switch (tree.getType()) {
+            case TRUE:
+                return addToken(new BooleanLiteralEvaluator(true), "true");
+            case FALSE:
+                return addToken(new BooleanLiteralEvaluator(false), "true");
+        }
+        throw new AttributeExpressionLanguageParsingException("Cannot build Boolean evaluator from tree " + tree.toString());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AttributeEvaluator.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AttributeEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AttributeEvaluator.java
deleted file mode 100644
index a0695a9..0000000
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AttributeEvaluator.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class AttributeEvaluator extends StringEvaluator {
-
-    private final Evaluator<String> nameEvaluator;
-
-    public AttributeEvaluator(final Evaluator<String> nameEvaluator) {
-        this.nameEvaluator = nameEvaluator;
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String nameValue = nameEvaluator.evaluate(attributes).getValue();
-        final String attributeValue = attributes.get(nameValue);
-        return new StringQueryResult(attributeValue);
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java
index ccf9d19..8c4213b 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java
@@ -70,4 +70,8 @@ public class AllAttributesEvaluator extends BooleanEvaluator implements Iteratin
     public Evaluator<?> getLogicEvaluator() {
         return booleanEvaluator;
     }
+
+    public MultiAttributeEvaluator getVariableIteratingEvaluator() {
+        return multiAttributeEvaluator;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java
index 6e8d485..eac571b 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java
@@ -70,4 +70,8 @@ public class AnyAttributeEvaluator extends BooleanEvaluator implements Iterating
     public Evaluator<Boolean> getLogicEvaluator() {
         return booleanEvaluator;
     }
+
+    public MultiAttributeEvaluator getVariableIteratingEvaluator() {
+        return multiAttributeEvaluator;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AttributeEvaluator.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AttributeEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AttributeEvaluator.java
new file mode 100644
index 0000000..0da063e
--- /dev/null
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AttributeEvaluator.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.attribute.expression.language.evaluation.selection;
+
+import java.util.Map;
+
+import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
+import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
+
+public class AttributeEvaluator extends StringEvaluator {
+
+    private final Evaluator<String> nameEvaluator;
+
+    public AttributeEvaluator(final Evaluator<String> nameEvaluator) {
+        this.nameEvaluator = nameEvaluator;
+    }
+
+    @Override
+    public QueryResult<String> evaluate(final Map<String, String> attributes) {
+        final String nameValue = nameEvaluator.evaluate(attributes).getValue();
+        final String attributeValue = attributes.get(nameValue);
+        return new StringQueryResult(attributeValue);
+    }
+
+    @Override
+    public Evaluator<?> getSubjectEvaluator() {
+        return null;
+    }
+
+    public Evaluator<String> getNameEvaluator() {
+        return nameEvaluator;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MappingEvaluator.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MappingEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MappingEvaluator.java
index e007a56..5f182c7 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MappingEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MappingEvaluator.java
@@ -68,4 +68,8 @@ public class MappingEvaluator<T> implements Evaluator<T> {
     public void setToken(final String token) {
         this.token = token;
     }
+
+    public MultiAttributeEvaluator getVariableIteratingEvaluator() {
+        return multiAttributeEvaluator;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java
index 509d7dd..cff9185 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java
@@ -66,4 +66,8 @@ public class MultiNamedAttributeEvaluator extends MultiAttributeEvaluator {
     public Evaluator<?> getLogicEvaluator() {
         return this;
     }
+
+    public List<String> getAttributeNames() {
+        return attributeNames;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
index 5946aa5..c4113c0 100644
--- a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
+++ b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
@@ -19,11 +19,11 @@ package org.apache.nifi.attribute.expression.language;
 import static java.lang.Double.NEGATIVE_INFINITY;
 import static java.lang.Double.NaN;
 import static java.lang.Double.POSITIVE_INFINITY;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.greaterThan;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
-import static org.hamcrest.Matchers.greaterThan;
-import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.Assert.fail;
 
 import java.io.BufferedInputStream;
@@ -40,6 +40,7 @@ import java.util.List;
 import java.util.Locale;
 import java.util.Map;
 
+import org.antlr.runtime.tree.Tree;
 import org.apache.nifi.attribute.expression.language.Query.Range;
 import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
@@ -47,14 +48,10 @@ import org.apache.nifi.attribute.expression.language.exception.AttributeExpressi
 import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
 import org.apache.nifi.flowfile.FlowFile;
-import org.antlr.runtime.tree.Tree;
-
 import org.apache.nifi.registry.VariableRegistry;
 import org.junit.Assert;
-
 import org.junit.Ignore;
 import org.junit.Test;
-
 import org.mockito.Mockito;
 
 public class TestQuery {
@@ -304,7 +301,7 @@ public class TestQuery {
 
     @Test(expected = AttributeExpressionLanguageException.class)
     public void testCannotCombineWithNonReducingFunction() {
-        Query.compileTree("${allAttributes( 'a.1' ):plus(1)}");
+        Query.compile("${allAttributes( 'a.1' ):plus(1)}");
     }
 
     @Test
@@ -357,6 +354,7 @@ public class TestQuery {
         assertEquals("Val", evaluateQueryForEscape("${attr:replaceAll(\"My (Val)ue{1,2}\", '$1')}", attributes));
     }
 
+    @SuppressWarnings("unchecked")
     private String evaluateQueryForEscape(final String queryString, final Map<String, String> attributes) {
         final FlowFile mockFlowFile = Mockito.mock(FlowFile.class);
         Mockito.when(mockFlowFile.getAttributes()).thenReturn(attributes);
@@ -639,7 +637,7 @@ public class TestQuery {
         final String query = "${ abc:equals('abc'):or( \n\t${xx:isNull()}\n) }";
         assertEquals(ResultType.BOOLEAN, Query.getResultType(query));
         Query.validateExpression(query, false);
-        assertEquals("true", Query.evaluateExpressions(query, Collections.EMPTY_MAP));
+        assertEquals("true", Query.evaluateExpressions(query, Collections.emptyMap()));
     }
 
     @Test
@@ -1675,25 +1673,25 @@ public class TestQuery {
         verifyEquals("${string:escapeHtml4()}", attributes, "special &clubs;");
       }
 
-      @Test
-      public void testUnescapeFunctions() {
-          final Map<String, String> attributes = new HashMap<>();
+    @Test
+    public void testUnescapeFunctions() {
+        final Map<String, String> attributes = new HashMap<>();
 
-          attributes.put("string", "making air \\\"QUOTES\\\".");
-          verifyEquals("${string:unescapeJson()}", attributes, "making air \"QUOTES\".");
+        attributes.put("string", "making air \\\"QUOTES\\\".");
+        verifyEquals("${string:unescapeJson()}", attributes, "making air \"QUOTES\".");
 
-          attributes.put("string", "M &amp; M");
-          verifyEquals("${string:unescapeXml()}", attributes, "M & M");
+        attributes.put("string", "M &amp; M");
+        verifyEquals("${string:unescapeXml()}", attributes, "M & M");
 
-          attributes.put("string", "\"making air \"\"QUOTES\"\".\"");
-          verifyEquals("${string:unescapeCsv()}", attributes, "making air \"QUOTES\".");
+        attributes.put("string", "\"making air \"\"QUOTES\"\".\"");
+        verifyEquals("${string:unescapeCsv()}", attributes, "making air \"QUOTES\".");
 
-          attributes.put("string", "special &iexcl;");
-          verifyEquals("${string:unescapeHtml3()}", attributes, "special ¡");
+        attributes.put("string", "special &iexcl;");
+        verifyEquals("${string:unescapeHtml3()}", attributes, "special ¡");
 
-          attributes.put("string", "special &clubs;");
-          verifyEquals("${string:unescapeHtml4()}", attributes, "special ♣");
-        }
+        attributes.put("string", "special &clubs;");
+        verifyEquals("${string:unescapeHtml4()}", attributes, "special ♣");
+    }
 
     @Test
     public void testIfElse() {
@@ -1709,9 +1707,9 @@ public class TestQuery {
         verifyEquals("${attr2:isNull():ifElse('a', 'b')}", attributes, "a");
         verifyEquals("${literal(true):ifElse('a', 'b')}", attributes, "a");
         verifyEquals("${literal(true):ifElse(false, 'b')}", attributes, "false");
-
     }
 
+
     private void verifyEquals(final String expression, final Map<String, String> attributes, final Object expectedResult) {
         verifyEquals(expression,attributes, null, expectedResult);
     }


[6/9] nifi git commit: NIFI-4224: - Initial implementation of Process Group level Variable Registry - Updated to incorporate PR Feedback - Changed log message because slf4j-simple apparently has a memory leak; passing a String instead of passing in the C

Posted by mc...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java
index 5acba8d..ea7cb96 100644
--- a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java
+++ b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java
@@ -17,9 +17,13 @@
 package org.apache.nifi.attribute.expression.language;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
 import org.junit.Ignore;
@@ -46,6 +50,7 @@ public class TestStandardPreparedQuery {
     }
 
     @Test
+    @Ignore("Intended for manual performance testing; should not be run in an automated environment")
     public void test10MIterations() {
         final Map<String, String> attrs = new HashMap<>();
         attrs.put("xx", "world");
@@ -84,6 +89,42 @@ public class TestStandardPreparedQuery {
 
     }
 
+    @Test
+    public void testVariableImpacted() {
+        final Set<String> attr = new HashSet<>();
+        attr.add("attr");
+
+        final Set<String> attr2 = new HashSet<>();
+        attr2.add("attr");
+        attr2.add("attr2");
+
+        final Set<String> abc = new HashSet<>();
+        abc.add("a");
+        abc.add("b");
+        abc.add("c");
+
+        assertTrue(Query.prepare("${attr}").getVariableImpact().isImpacted("attr"));
+        assertFalse(Query.prepare("${attr}").getVariableImpact().isImpacted("attr2"));
+        assertTrue(Query.prepare("${attr:trim():toUpper():equals('abc')}").getVariableImpact().isImpacted("attr"));
+
+        assertFalse(Query.prepare("${anyAttribute('a', 'b', 'c'):equals('hello')}").getVariableImpact().isImpacted("attr"));
+        assertTrue(Query.prepare("${anyAttribute('a', 'b', 'c'):equals('hello')}").getVariableImpact().isImpacted("a"));
+        assertTrue(Query.prepare("${anyAttribute('a', 'b', 'c'):equals('hello')}").getVariableImpact().isImpacted("b"));
+        assertTrue(Query.prepare("${anyAttribute('a', 'b', 'c'):equals('hello')}").getVariableImpact().isImpacted("c"));
+
+        assertFalse(Query.prepare("${allAttributes('a', 'b', 'c'):equals('hello')}").getVariableImpact().isImpacted("attr"));
+        assertTrue(Query.prepare("${allAttributes('a', 'b', 'c'):equals('hello')}").getVariableImpact().isImpacted("a"));
+        assertTrue(Query.prepare("${allAttributes('a', 'b', 'c'):equals('hello')}").getVariableImpact().isImpacted("b"));
+        assertTrue(Query.prepare("${allAttributes('a', 'b', 'c'):equals('hello')}").getVariableImpact().isImpacted("c"));
+
+        assertTrue(Query.prepare("${attr:equals('${attr2}')}").getVariableImpact().isImpacted("attr"));
+        assertTrue(Query.prepare("${attr:equals('${attr2}')}").getVariableImpact().isImpacted("attr2"));
+        assertFalse(Query.prepare("${attr:equals('${attr2}')}").getVariableImpact().isImpacted("attr3"));
+
+        assertTrue(Query.prepare("${allMatchingAttributes('a.*'):equals('hello')}").getVariableImpact().isImpacted("attr"));
+        assertTrue(Query.prepare("${anyMatchingAttribute('a.*'):equals('hello')}").getVariableImpact().isImpacted("attr"));
+    }
+
     private String evaluate(final String query, final Map<String, String> attrs) {
         final String evaluated = ((StandardPreparedQuery) Query.prepare(query)).evaluateExpressions(attrs, null);
         return evaluated;

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestValueLookup.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestValueLookup.java b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestValueLookup.java
index ec8ba66..1010ac7 100644
--- a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestValueLookup.java
+++ b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestValueLookup.java
@@ -25,12 +25,10 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertEquals;
 
-/**
- *
- */
 public class TestValueLookup {
 
     @Test
+    @SuppressWarnings("unchecked")
     public void testCreateCustomVariableRegistry() {
 
         final VariableRegistry variableRegistry = VariableRegistry.ENVIRONMENT_SYSTEM_REGISTRY;

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/AffectedComponentDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/AffectedComponentDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/AffectedComponentDTO.java
new file mode 100644
index 0000000..5d631ed
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/AffectedComponentDTO.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.web.api.dto;
+
+import javax.xml.bind.annotation.XmlType;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
+@XmlType(name = "affectedComponent")
+public class AffectedComponentDTO {
+    public static final String COMPONENT_TYPE_PROCESSOR = "PROCESSOR";
+    public static final String COMPONENT_TYPE_CONTROLLER_SERVICE = "CONTROLLER_SERVICE";
+
+    private String parentGroupId;
+    private String componentId;
+    private String componentType;
+
+    @ApiModelProperty("The UUID of the Process Group that this component is in")
+    public String getParentGroupId() {
+        return parentGroupId;
+    }
+
+    public void setParentGroupId(final String parentGroupId) {
+        this.parentGroupId = parentGroupId;
+    }
+
+    @ApiModelProperty("The UUID of this component")
+    public String getComponentId() {
+        return componentId;
+    }
+
+    public void setComponentId(final String componentId) {
+        this.componentId = componentId;
+    }
+
+    @ApiModelProperty(value = "The type of this component", allowableValues = COMPONENT_TYPE_PROCESSOR + "," + COMPONENT_TYPE_CONTROLLER_SERVICE)
+    public String getComponentType() {
+        return componentType;
+    }
+
+    public void setComponentType(final String componentType) {
+        this.componentType = componentType;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.java
index 42d605d..f4b6f31 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.java
@@ -17,6 +17,9 @@
 package org.apache.nifi.web.api.dto;
 
 import com.wordnik.swagger.annotations.ApiModelProperty;
+
+import java.util.Map;
+
 import javax.xml.bind.annotation.XmlType;
 
 /**
@@ -27,6 +30,7 @@ public class ProcessGroupDTO extends ComponentDTO {
 
     private String name;
     private String comments;
+    private Map<String, String> variables;
 
     private Integer runningCount;
     private Integer stoppedCount;
@@ -200,4 +204,16 @@ public class ProcessGroupDTO extends ComponentDTO {
         this.inactiveRemotePortCount = inactiveRemotePortCount;
     }
 
+
+    @ApiModelProperty(value = "The variables that are configured for the Process Group. Note that this map contains only "
+        + "those variables that are defined on this Process Group and not any variables that are defined in the parent "
+        + "Process Group, etc. I.e., this Map will not contain all variables that are accessible by components in this "
+        + "Process Group by rather only the variables that are defined for this Process Group itself.", readOnly = true)
+    public Map<String, String> getVariables() {
+        return variables;
+    }
+
+    public void setVariables(final Map<String, String> variables) {
+        this.variables = variables;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableDTO.java
new file mode 100644
index 0000000..c686316
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableDTO.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.web.api.dto;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import javax.xml.bind.annotation.XmlType;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
+@XmlType(name = "variable")
+public class VariableDTO {
+    private String name;
+    private String value;
+    private String processGroupId;
+    private Set<AffectedComponentDTO> affectedComponents = new HashSet<>();
+
+    @ApiModelProperty("The name of the variable")
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    @ApiModelProperty("The value of the variable")
+    public String getValue() {
+        return value;
+    }
+
+    public void setValue(String value) {
+        this.value = value;
+    }
+
+    @ApiModelProperty(value = "The ID of the Process Group where this Variable is defined", readOnly = true)
+    public String getProcessGroupId() {
+        return processGroupId;
+    }
+
+    public void setProcessGroupId(String groupId) {
+        this.processGroupId = groupId;
+    }
+
+    @ApiModelProperty(value = "A set of all components that will be affected if the value of this variable is changed", readOnly = true)
+    public Set<AffectedComponentDTO> getAffectedComponents() {
+        return affectedComponents;
+    }
+
+    public void setAffectedComponents(Set<AffectedComponentDTO> affectedComponents) {
+        this.affectedComponents = affectedComponents;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryDTO.java
new file mode 100644
index 0000000..c106a9a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryDTO.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.web.api.dto;
+
+import java.util.Set;
+
+import javax.xml.bind.annotation.XmlType;
+
+import org.apache.nifi.web.api.entity.VariableEntity;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
+@XmlType(name = "variableRegistry")
+public class VariableRegistryDTO {
+    private Set<VariableEntity> variables;
+    private String groupId;
+
+    public void setVariables(final Set<VariableEntity> variables) {
+        this.variables = variables;
+    }
+
+    @ApiModelProperty("The variables that are available in this Variable Registry")
+    public Set<VariableEntity> getVariables() {
+        return variables;
+    }
+
+    public void setProcessGroupId(final String groupId) {
+        this.groupId = groupId;
+    }
+
+    @ApiModelProperty("The UUID of the Process Group that this Variable Registry belongs to")
+    public String getProcessGroupId() {
+        return groupId;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateRequestDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateRequestDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateRequestDTO.java
new file mode 100644
index 0000000..06a0dc2
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateRequestDTO.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.web.api.dto;
+
+import java.util.Date;
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlType;
+import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
+
+import org.apache.nifi.web.api.dto.util.TimestampAdapter;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
+@XmlType(name = "variableRegistryUpdateRequest")
+public class VariableRegistryUpdateRequestDTO {
+    private String requestId;
+    private String processGroupId;
+    private String uri;
+    private Date submissionTime = new Date();
+    private Date lastUpdated = new Date();
+    private boolean complete = false;
+    private String failureReason;
+    private List<VariableRegistryUpdateStepDTO> updateSteps;
+
+
+    @ApiModelProperty("The unique ID of the Process Group that the variable registry belongs to")
+    public String getProcessGroupId() {
+        return processGroupId;
+    }
+
+    public void setProcessGroupId(String processGroupId) {
+        this.processGroupId = processGroupId;
+    }
+
+    @ApiModelProperty(value = "The unique ID of this request.", readOnly = true)
+    public String getRequestId() {
+        return requestId;
+    }
+
+    public void setRequestId(String requestId) {
+        this.requestId = requestId;
+    }
+
+    @ApiModelProperty(value = "The URI for future requests to this drop request.", readOnly = true)
+    public String getUri() {
+        return uri;
+    }
+
+    public void setUri(String uri) {
+        this.uri = uri;
+    }
+
+    @XmlJavaTypeAdapter(TimestampAdapter.class)
+    @ApiModelProperty(value = "The time at which this request was submitted.", dataType = "string", readOnly = true)
+    public Date getSubmissionTime() {
+        return submissionTime;
+    }
+
+    public void setSubmissionTime(Date submissionTime) {
+        this.submissionTime = submissionTime;
+    }
+
+    @XmlJavaTypeAdapter(TimestampAdapter.class)
+    @ApiModelProperty(value = "The last time this request was updated.", dataType = "string", readOnly = true)
+    public Date getLastUpdated() {
+        return lastUpdated;
+    }
+
+    public void setLastUpdated(Date lastUpdated) {
+        this.lastUpdated = lastUpdated;
+    }
+
+    @ApiModelProperty(value = "The steps that are required in order to complete the request, along with the status of each", readOnly = true)
+    public List<VariableRegistryUpdateStepDTO> getUpdateSteps() {
+        return updateSteps;
+    }
+
+    public void setUpdateSteps(List<VariableRegistryUpdateStepDTO> updateSteps) {
+        this.updateSteps = updateSteps;
+    }
+
+    @ApiModelProperty(value = "Whether or not this request has completed", readOnly = true)
+    public boolean isComplete() {
+        return complete;
+    }
+
+    public void setComplete(boolean complete) {
+        this.complete = complete;
+    }
+
+    @ApiModelProperty(value = "An explanation of why this request failed, or null if this request has not failed", readOnly = true)
+    public String getFailureReason() {
+        return failureReason;
+    }
+
+    public void setFailureReason(String reason) {
+        this.failureReason = reason;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateStepDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateStepDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateStepDTO.java
new file mode 100644
index 0000000..e1c8cee
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateStepDTO.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.web.api.dto;
+
+import javax.xml.bind.annotation.XmlType;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
+@XmlType(name = "varaibleRegistryUpdateStep")
+public class VariableRegistryUpdateStepDTO {
+    private String description;
+    private boolean complete;
+    private String failureReason;
+
+    public VariableRegistryUpdateStepDTO() {
+    }
+
+    @ApiModelProperty(value = "Explanation of what happens in this step", readOnly = true)
+    public String getDescription() {
+        return description;
+    }
+
+    public void setDescription(String description) {
+        this.description = description;
+    }
+
+    @ApiModelProperty(value = "Whether or not this step has completed", readOnly = true)
+    public boolean isComplete() {
+        return complete;
+    }
+
+    public void setComplete(boolean complete) {
+        this.complete = complete;
+    }
+
+    @ApiModelProperty(value = "An explanation of why this step failed, or null if this step did not fail", readOnly = true)
+    public String getFailureReason() {
+        return failureReason;
+    }
+
+    public void setFailureReason(String reason) {
+        this.failureReason = reason;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ActivateControllerServicesEntity.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ActivateControllerServicesEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ActivateControllerServicesEntity.java
new file mode 100644
index 0000000..a58f821
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ActivateControllerServicesEntity.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.api.entity;
+
+import java.util.Map;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.nifi.web.api.dto.RevisionDTO;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
+@XmlRootElement(name = "activateControllerServicesEntity")
+public class ActivateControllerServicesEntity extends Entity {
+    public static final String STATE_ENABLED = "ENABLED";
+    public static final String STATE_DISABLED = "DISABLED";
+
+    private String id;
+    private String state;
+    private Map<String, RevisionDTO> components;
+
+    @ApiModelProperty("The id of the ProcessGroup")
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    /**
+     * @return The desired state of the descendant components. Possible states are 'RUNNING' and 'STOPPED'
+     */
+    @ApiModelProperty(value = "The desired state of the descendant components",
+        allowableValues = STATE_ENABLED + ", " + STATE_DISABLED)
+    public String getState() {
+        return state;
+    }
+
+    public void setState(String state) {
+        this.state = state;
+    }
+
+    @ApiModelProperty("Optional services to schedule. If not specified, all authorized descendant controller services will be used.")
+    public Map<String, RevisionDTO> getComponents() {
+        return components;
+    }
+
+    public void setComponents(Map<String, RevisionDTO> components) {
+        this.components = components;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ScheduleComponentsEntity.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ScheduleComponentsEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ScheduleComponentsEntity.java
index 9aeef40..3e4c9a4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ScheduleComponentsEntity.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ScheduleComponentsEntity.java
@@ -16,17 +16,21 @@
  */
 package org.apache.nifi.web.api.entity;
 
-import com.wordnik.swagger.annotations.ApiModelProperty;
-import org.apache.nifi.web.api.dto.RevisionDTO;
+import java.util.Map;
 
 import javax.xml.bind.annotation.XmlRootElement;
-import java.util.Map;
+
+import org.apache.nifi.web.api.dto.RevisionDTO;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
 
 /**
  * A serialized representation of this class can be placed in the entity body of a request to the API.
  */
 @XmlRootElement(name = "scheduleComponentEntity")
 public class ScheduleComponentsEntity extends Entity {
+    public static final String STATE_RUNNING = "RUNNING";
+    public static final String STATE_STOPPED = "STOPPED";
 
     private String id;
     private String state;
@@ -51,7 +55,7 @@ public class ScheduleComponentsEntity extends Entity {
      */
     @ApiModelProperty(
         value = "The desired state of the descendant components",
-        allowableValues = "RUNNING, STOPPED"
+        allowableValues = STATE_RUNNING + ", " + STATE_STOPPED
     )
     public String getState() {
         return state;

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VariableEntity.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VariableEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VariableEntity.java
new file mode 100644
index 0000000..06f6fcf
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VariableEntity.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.web.api.entity;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.nifi.web.api.dto.VariableDTO;
+import org.apache.nifi.web.api.dto.WritablePermission;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
+@XmlRootElement(name = "variableEntity")
+public class VariableEntity extends Entity implements WritablePermission {
+    private VariableDTO variable;
+    private Boolean canWrite;
+
+    @Override
+    @ApiModelProperty(value = "Indicates whether the user can write a given resource.", readOnly = true)
+    public Boolean getCanWrite() {
+        return canWrite;
+    }
+
+    @Override
+    public void setCanWrite(Boolean canWrite) {
+        this.canWrite = canWrite;
+    }
+
+    @ApiModelProperty("The variable information")
+    public VariableDTO getVariable() {
+        return variable;
+    }
+
+    public void setVariable(VariableDTO variable) {
+        this.variable = variable;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VariableRegistryEntity.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VariableRegistryEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VariableRegistryEntity.java
new file mode 100644
index 0000000..d876453
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VariableRegistryEntity.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.web.api.entity;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.nifi.web.api.dto.RevisionDTO;
+import org.apache.nifi.web.api.dto.VariableRegistryDTO;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
+@XmlRootElement(name = "variableRegistryEntity")
+public class VariableRegistryEntity extends Entity {
+    private RevisionDTO groupRevision;
+    private VariableRegistryDTO variableRegistry;
+
+
+    @ApiModelProperty("The Variable Registry.")
+    public VariableRegistryDTO getVariableRegistry() {
+        return variableRegistry;
+    }
+
+    public void setVariableRegistry(VariableRegistryDTO variableRegistry) {
+        this.variableRegistry = variableRegistry;
+    }
+
+    @ApiModelProperty("The revision of the Process Group that the Variable Registry belongs to")
+    public RevisionDTO getProcessGroupRevision() {
+        return groupRevision;
+    }
+
+    public void setProcessGroupRevision(RevisionDTO revision) {
+        this.groupRevision = revision;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VariableRegistryUpdateRequestEntity.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VariableRegistryUpdateRequestEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VariableRegistryUpdateRequestEntity.java
new file mode 100644
index 0000000..77257af
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VariableRegistryUpdateRequestEntity.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.web.api.entity;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.nifi.web.api.dto.RevisionDTO;
+import org.apache.nifi.web.api.dto.VariableRegistryUpdateRequestDTO;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
+@XmlRootElement(name = "variableRegistryUpdateRequestEntity")
+public class VariableRegistryUpdateRequestEntity extends Entity {
+    private VariableRegistryUpdateRequestDTO requestDto;
+    private RevisionDTO processGroupRevision;
+
+    @ApiModelProperty("The revision for the Process Group that owns this variable registry.")
+    public RevisionDTO getProcessGroupRevision() {
+        return processGroupRevision;
+    }
+
+    public void setProcessGroupRevision(RevisionDTO revision) {
+        this.processGroupRevision = revision;
+    }
+
+    @ApiModelProperty("The Variable Registry Update Request")
+    public VariableRegistryUpdateRequestDTO getRequestDto() {
+        return requestDto;
+    }
+
+    public void setRequestDto(VariableRegistryUpdateRequestDTO requestDto) {
+        this.requestDto = requestDto;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java
index 2fc55a4..c102746 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java
@@ -16,6 +16,16 @@
  */
 package org.apache.nifi.cluster.coordination.http;
 
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import javax.ws.rs.core.StreamingOutput;
+
 import org.apache.nifi.cluster.coordination.http.endpoints.AccessPolicyEndpointMerger;
 import org.apache.nifi.cluster.coordination.http.endpoints.BulletinBoardEndpointMerger;
 import org.apache.nifi.cluster.coordination.http.endpoints.ComponentStateEndpointMerger;
@@ -77,15 +87,6 @@ import org.apache.nifi.util.NiFiProperties;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.ws.rs.core.StreamingOutput;
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
-
 public class StandardHttpResponseMapper implements HttpResponseMapper {
 
     private Logger logger = LoggerFactory.getLogger(StandardHttpResponseMapper.class);

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java
index 45c6ead..ff5a8af 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java
@@ -16,24 +16,6 @@
  */
 package org.apache.nifi.controller;
 
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
-import org.apache.nifi.bundle.Bundle;
-import org.apache.nifi.bundle.BundleCoordinate;
-import org.apache.nifi.components.ConfigurableComponent;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.ValidationContext;
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.controller.service.ControllerServiceNode;
-import org.apache.nifi.controller.service.ControllerServiceProvider;
-import org.apache.nifi.nar.ExtensionManager;
-import org.apache.nifi.nar.NarCloseable;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.util.CharacterFilterUtils;
-import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import java.net.MalformedURLException;
 import java.net.URL;
 import java.util.ArrayList;
@@ -53,6 +35,24 @@ import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.function.Consumer;
 
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
+import org.apache.nifi.bundle.Bundle;
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.service.ControllerServiceNode;
+import org.apache.nifi.controller.service.ControllerServiceProvider;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.NarCloseable;
+import org.apache.nifi.registry.ComponentVariableRegistry;
+import org.apache.nifi.util.CharacterFilterUtils;
+import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 public abstract class AbstractConfiguredComponent implements ConfigurableComponent, ConfiguredComponent {
     private static final Logger logger = LoggerFactory.getLogger(AbstractConfiguredComponent.class);
 
@@ -64,7 +64,7 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone
     private final AtomicReference<ValidationContext> validationContext = new AtomicReference<>();
     private final String componentType;
     private final String componentCanonicalClass;
-    private final VariableRegistry variableRegistry;
+    private final ComponentVariableRegistry variableRegistry;
     private final ReloadComponent reloadComponent;
 
     private final AtomicBoolean isExtensionMissing;
@@ -74,7 +74,7 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone
 
     public AbstractConfiguredComponent(final String id,
                                        final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider,
-                                       final String componentType, final String componentCanonicalClass, final VariableRegistry variableRegistry,
+                                       final String componentType, final String componentCanonicalClass, final ComponentVariableRegistry variableRegistry,
                                        final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
         this.id = id;
         this.validationContextFactory = validationContextFactory;
@@ -541,7 +541,7 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone
         }
     }
 
-    protected VariableRegistry getVariableRegistry() {
+    public ComponentVariableRegistry getVariableRegistry() {
         return this.variableRegistry;
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java
index 069e6ce..940ac21 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java
@@ -29,6 +29,7 @@ import org.apache.nifi.components.ConfigurableComponent;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.registry.ComponentVariableRegistry;
 
 import java.net.URL;
 import java.util.Collection;
@@ -38,6 +39,7 @@ import java.util.Set;
 
 public interface ConfiguredComponent extends ComponentAuthorizable {
 
+    @Override
     public String getIdentifier();
 
     public String getName();
@@ -99,6 +101,11 @@ public interface ConfiguredComponent extends ComponentAuthorizable {
      */
     boolean isDeprecated();
 
+    /**
+     * @return the variable registry for this component
+     */
+    ComponentVariableRegistry getVariableRegistry();
+
     @Override
     default AuthorizationResult checkAuthorization(Authorizer authorizer, RequestAction action, NiFiUser user, Map<String, String> resourceContext) {
         // if this is a modification request and the reporting task is restricted ensure the user has elevated privileges. if this

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java
index 5bb8981..c6f30b5 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java
@@ -18,6 +18,7 @@ package org.apache.nifi.controller;
 
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Future;
 
 import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Funnel;
@@ -41,7 +42,7 @@ public interface ProcessScheduler {
      * @param procNode to start
      * @throws IllegalStateException if the Processor is disabled
      */
-    void startProcessor(ProcessorNode procNode);
+    Future<Void> startProcessor(ProcessorNode procNode);
 
     /**
      * Stops scheduling the given processor to run and invokes all methods on
@@ -52,7 +53,7 @@ public interface ProcessScheduler {
      *
      * @param procNode to stop
      */
-    void stopProcessor(ProcessorNode procNode);
+    Future<Void> stopProcessor(ProcessorNode procNode);
 
     /**
      * Starts scheduling the given Port to run. If the Port is already scheduled
@@ -169,12 +170,12 @@ public interface ProcessScheduler {
      * Disables all of the given Controller Services in the order provided by the List
      * @param services the controller services to disable
      */
-    void disableControllerServices(List<ControllerServiceNode> services);
+    CompletableFuture<Void> disableControllerServices(List<ControllerServiceNode> services);
 
     /**
      * Disables the Controller Service so that it can be updated
      *
      * @param service to disable
      */
-    void disableControllerService(ControllerServiceNode service);
+    CompletableFuture<Void> disableControllerService(ControllerServiceNode service);
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java
index aac5e52..ba2e59b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java
@@ -16,6 +16,13 @@
  */
 package org.apache.nifi.controller;
 
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
 import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
 import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.controller.scheduling.ScheduleState;
@@ -26,18 +33,12 @@ import org.apache.nifi.logging.LogLevel;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.Processor;
 import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.ComponentVariableRegistry;
 import org.apache.nifi.scheduling.ExecutionNode;
 import org.apache.nifi.scheduling.SchedulingStrategy;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-
 public abstract class ProcessorNode extends AbstractConfiguredComponent implements Connectable {
 
     private static final Logger logger = LoggerFactory.getLogger(ProcessorNode.class);
@@ -46,7 +47,7 @@ public abstract class ProcessorNode extends AbstractConfiguredComponent implemen
 
     public ProcessorNode(final String id,
                          final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider,
-                         final String componentType, final String componentCanonicalClass, final VariableRegistry variableRegistry,
+                         final String componentType, final String componentCanonicalClass, final ComponentVariableRegistry variableRegistry,
                          final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
         super(id, validationContextFactory, serviceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent, isExtensionMissing);
         this.scheduledState = new AtomicReference<>(ScheduledState.STOPPED);
@@ -185,7 +186,7 @@ public abstract class ProcessorNode extends AbstractConfiguredComponent implemen
      *            the ScheduleState that can be used to ensure that the running state (STOPPED, RUNNING, etc.)
      *            as well as the active thread counts are kept in sync
      */
-    public abstract <T extends ProcessContext & ControllerServiceLookup> void stop(ScheduledExecutorService scheduler,
+    public abstract <T extends ProcessContext & ControllerServiceLookup> CompletableFuture<Void> stop(ScheduledExecutorService scheduler,
         T processContext, SchedulingAgent schedulingAgent, ScheduleState scheduleState);
 
     /**

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java
index faf530f..3dd1076 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java
@@ -108,7 +108,7 @@ public interface ControllerServiceNode extends ConfiguredComponent {
      *            implementation of {@link ScheduledExecutorService} used to
      *            initiate service disabling task
      */
-    void disable(ScheduledExecutorService scheduler);
+    CompletableFuture<Void> disable(ScheduledExecutorService scheduler);
 
     /**
      * @return the ControllerServiceReference that describes which components are referencing this Controller Service

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java
index f7ba5e5..010ecdf 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java
@@ -19,7 +19,7 @@ package org.apache.nifi.controller.service;
 import java.net.URL;
 import java.util.Collection;
 import java.util.Set;
-import java.util.concurrent.Future;
+import java.util.concurrent.CompletableFuture;
 
 import org.apache.nifi.annotation.lifecycle.OnAdded;
 import org.apache.nifi.bundle.BundleCoordinate;
@@ -72,7 +72,7 @@ public interface ControllerServiceProvider extends ControllerServiceLookup {
      * @param serviceNode the service node
      * @return a Future that can be used to wait for the service to finish being enabled.
      */
-    Future<Void> enableControllerService(ControllerServiceNode serviceNode);
+    CompletableFuture<Void> enableControllerService(ControllerServiceNode serviceNode);
 
     /**
      * Enables the collection of services. If a service in this collection
@@ -90,7 +90,7 @@ public interface ControllerServiceProvider extends ControllerServiceLookup {
      *
      * @param serviceNode the node
      */
-    void disableControllerService(ControllerServiceNode serviceNode);
+    CompletableFuture<Void> disableControllerService(ControllerServiceNode serviceNode);
 
     /**
      * @return a Set of all Controller Services that exist for this service

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java
index df18c62..13c5844 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.controller.service;
 
+import java.util.List;
 import java.util.Set;
 
 import org.apache.nifi.controller.ConfiguredComponent;
@@ -43,4 +44,13 @@ public interface ControllerServiceReference {
      * Controller Services)
      */
     Set<ConfiguredComponent> getActiveReferences();
+
+    /**
+     * Returns a List of all components that reference this Controller Service (recursively) that
+     * are of the given type
+     *
+     * @param componentType the type of component that is desirable
+     * @return a List of all components that reference this Controller Service that are of the given type
+     */
+    <T> List<T> findRecursiveReferences(Class<T> componentType);
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
index 122e454..bf789f7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
@@ -16,12 +16,20 @@
  */
 package org.apache.nifi.groups;
 
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Predicate;
+
 import org.apache.nifi.authorization.resource.ComponentAuthorizable;
 import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.connectable.Positionable;
+import org.apache.nifi.controller.ConfiguredComponent;
 import org.apache.nifi.controller.ProcessorNode;
 import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.Snippet;
@@ -30,13 +38,9 @@ import org.apache.nifi.controller.label.Label;
 import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.Processor;
+import org.apache.nifi.registry.ComponentVariableRegistry;
 import org.apache.nifi.remote.RemoteGroupPort;
 
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-import java.util.function.Predicate;
-
 /**
  * <p>
  * ProcessGroup objects are containers for processing entities, such as
@@ -84,6 +88,7 @@ public interface ProcessGroup extends ComponentAuthorizable, Positionable {
     /**
      * @return the ID of the ProcessGroup
      */
+    @Override
     String getIdentifier();
 
     /**
@@ -159,7 +164,7 @@ public interface ProcessGroup extends ComponentAuthorizable, Positionable {
      * @throws IllegalStateException if the processor is not valid, or is
      * already running
      */
-    void startProcessor(ProcessorNode processor);
+    CompletableFuture<Void> startProcessor(ProcessorNode processor);
 
     /**
      * Starts the given Input Port
@@ -187,7 +192,7 @@ public interface ProcessGroup extends ComponentAuthorizable, Positionable {
      *
      * @param processor to stop
      */
-    void stopProcessor(ProcessorNode processor);
+    CompletableFuture<Void> stopProcessor(ProcessorNode processor);
 
     /**
      * Stops the given Port
@@ -814,6 +819,15 @@ public interface ProcessGroup extends ComponentAuthorizable, Positionable {
     void verifyCanMove(Snippet snippet, ProcessGroup newProcessGroup);
 
     /**
+     * Ensures that the given variables can be updated
+     *
+     * @param updatedVariables the new set of variable names and values
+     *
+     * @throws IllegalStateException if one or more variables that are listed cannot be updated at this time
+     */
+    void verifyCanUpdateVariables(Map<String, String> updatedVariables);
+
+    /**
      * Adds the given template to this Process Group
      *
      * @param template the template to add
@@ -853,4 +867,27 @@ public interface ProcessGroup extends ComponentAuthorizable, Positionable {
      * @return a Set of all Templates that belong to this Process Group and any descendant Process Groups
      */
     Set<Template> findAllTemplates();
+
+    /**
+     * Updates the variables that are provided by this Process Group
+     *
+     * @param variables the variables to provide
+     * @throws IllegalStateException if the Process Group is not in a state that allows the variables to be updated
+     */
+    void setVariables(Map<String, String> variables);
+
+    /**
+     * Returns the Variable Registry for this Process Group
+     *
+     * @return the Variable Registry for this Process Group
+     */
+    ComponentVariableRegistry getVariableRegistry();
+
+    /**
+     * Returns a set of all components that are affected by the variable with the given name
+     *
+     * @param variableName the name of the variable
+     * @return a set of all components that are affected by the variable with the given name
+     */
+    Set<ConfiguredComponent> getComponentsAffectedByVariable(String variableName);
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index 2e351a3..94645cf 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -34,9 +34,9 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.Future;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
@@ -195,7 +195,10 @@ import org.apache.nifi.provenance.ProvenanceEventRecord;
 import org.apache.nifi.provenance.ProvenanceEventType;
 import org.apache.nifi.provenance.ProvenanceRepository;
 import org.apache.nifi.provenance.StandardProvenanceEventRecord;
+import org.apache.nifi.registry.ComponentVariableRegistry;
 import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.variable.MutableVariableRegistry;
+import org.apache.nifi.registry.variable.StandardComponentVariableRegistry;
 import org.apache.nifi.remote.HttpRemoteSiteListener;
 import org.apache.nifi.remote.RemoteGroupPort;
 import org.apache.nifi.remote.RemoteResourceManager;
@@ -495,15 +498,15 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
             throw new RuntimeException(e);
         }
 
-        processScheduler = new StandardProcessScheduler(this, encryptor, stateManagerProvider, this.variableRegistry, this.nifiProperties);
+        processScheduler = new StandardProcessScheduler(this, encryptor, stateManagerProvider, this.nifiProperties);
         eventDrivenWorkerQueue = new EventDrivenWorkerQueue(false, false, processScheduler);
 
         final ProcessContextFactory contextFactory = new ProcessContextFactory(contentRepository, flowFileRepository, flowFileEventRepository, counterRepositoryRef.get(), provenanceRepository);
         processScheduler.setSchedulingAgent(SchedulingStrategy.EVENT_DRIVEN, new EventDrivenSchedulingAgent(
-                eventDrivenEngineRef.get(), this, stateManagerProvider, eventDrivenWorkerQueue, contextFactory, maxEventDrivenThreads.get(), encryptor, this.variableRegistry));
+            eventDrivenEngineRef.get(), this, stateManagerProvider, eventDrivenWorkerQueue, contextFactory, maxEventDrivenThreads.get(), encryptor));
 
-        final QuartzSchedulingAgent quartzSchedulingAgent = new QuartzSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor, this.variableRegistry);
-        final TimerDrivenSchedulingAgent timerDrivenAgent = new TimerDrivenSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor, this.variableRegistry, this.nifiProperties);
+        final QuartzSchedulingAgent quartzSchedulingAgent = new QuartzSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor);
+        final TimerDrivenSchedulingAgent timerDrivenAgent = new TimerDrivenSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor, this.nifiProperties);
         processScheduler.setSchedulingAgent(SchedulingStrategy.TIMER_DRIVEN, timerDrivenAgent);
         // PRIMARY_NODE_ONLY is deprecated, but still exists to handle processors that are still defined with it (they haven't been re-configured with executeNode = PRIMARY).
         processScheduler.setSchedulingAgent(SchedulingStrategy.PRIMARY_NODE_ONLY, timerDrivenAgent);
@@ -540,7 +543,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         this.snippetManager = new SnippetManager();
 
         final ProcessGroup rootGroup = new StandardProcessGroup(ComponentIdGenerator.generateId().toString(), this, processScheduler,
-                nifiProperties, encryptor, this, this.variableRegistry);
+            nifiProperties, encryptor, this, new MutableVariableRegistry(this.variableRegistry));
         rootGroup.setName(DEFAULT_ROOT_GROUP_NAME);
         setRootGroup(rootGroup);
         instanceId = ComponentIdGenerator.generateId().toString();
@@ -1022,7 +1025,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
      * @throws NullPointerException if the argument is null
      */
     public ProcessGroup createProcessGroup(final String id) {
-        return new StandardProcessGroup(requireNonNull(id).intern(), this, processScheduler, nifiProperties, encryptor, this, variableRegistry);
+        return new StandardProcessGroup(requireNonNull(id).intern(), this, processScheduler, nifiProperties, encryptor, this, new MutableVariableRegistry(variableRegistry));
     }
 
     /**
@@ -1099,15 +1102,17 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
             creationSuccessful = false;
         }
 
-        final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider, variableRegistry);
+        final ComponentVariableRegistry componentVarRegistry = new StandardComponentVariableRegistry(this.variableRegistry);
+        final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider, componentVarRegistry);
         final ProcessorNode procNode;
         if (creationSuccessful) {
-            procNode = new StandardProcessorNode(processor, id, validationContextFactory, processScheduler, controllerServiceProvider, nifiProperties, variableRegistry, this);
+            procNode = new StandardProcessorNode(processor, id, validationContextFactory, processScheduler, controllerServiceProvider,
+                nifiProperties, componentVarRegistry, this);
         } else {
             final String simpleClassName = type.contains(".") ? StringUtils.substringAfterLast(type, ".") : type;
             final String componentType = "(Missing) " + simpleClassName;
-            procNode = new StandardProcessorNode(
-                    processor, id, validationContextFactory, processScheduler, controllerServiceProvider, componentType, type, nifiProperties, variableRegistry, this, true);
+            procNode = new StandardProcessorNode(processor, id, validationContextFactory, processScheduler, controllerServiceProvider,
+                componentType, type, nifiProperties, componentVarRegistry, this, true);
         }
 
         final LogRepository logRepository = LogRepositoryFactory.getRepository(id);
@@ -1223,7 +1228,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         // call OnRemoved for the existing processor using the previous instance class loader
         try (final NarCloseable x = NarCloseable.withComponentNarLoader(existingInstanceClassLoader)) {
             final StandardProcessContext processContext = new StandardProcessContext(
-                    existingNode, controllerServiceProvider, encryptor, getStateManagerProvider().getStateManager(id), variableRegistry);
+                existingNode, controllerServiceProvider, encryptor, getStateManagerProvider().getStateManager(id));
             ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, existingNode.getProcessor(), processContext);
         } finally {
             ExtensionManager.closeURLClassLoader(id, existingInstanceClassLoader);
@@ -1943,6 +1948,10 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
                 childGroup.setPosition(toPosition(groupDTO.getPosition()));
                 childGroup.setComments(groupDTO.getComments());
                 childGroup.setName(groupDTO.getName());
+                if (groupDTO.getVariables() != null) {
+                    childGroup.setVariables(groupDTO.getVariables());
+                }
+
                 group.addProcessGroup(childGroup);
 
                 final FlowSnippetDTO contents = groupDTO.getContents();
@@ -3124,15 +3133,16 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
             creationSuccessful = false;
         }
 
-        final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider, variableRegistry);
+        final ComponentVariableRegistry componentVarRegistry = new StandardComponentVariableRegistry(this.variableRegistry);
+        final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider, componentVarRegistry);
         final ReportingTaskNode taskNode;
         if (creationSuccessful) {
-            taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, variableRegistry, this);
+            taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, componentVarRegistry, this);
         } else {
             final String simpleClassName = type.contains(".") ? StringUtils.substringAfterLast(type, ".") : type;
             final String componentType = "(Missing) " + simpleClassName;
 
-            taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, componentType, type, variableRegistry, this, true);
+            taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, componentType, type, componentVarRegistry, this, true);
         }
 
         taskNode.setName(taskNode.getReportingTask().getClass().getSimpleName());
@@ -3395,7 +3405,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
     }
 
     @Override
-    public Future<Void> enableControllerService(final ControllerServiceNode serviceNode) {
+    public CompletableFuture<Void> enableControllerService(final ControllerServiceNode serviceNode) {
         return controllerServiceProvider.enableControllerService(serviceNode);
     }
 
@@ -3405,9 +3415,9 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
     }
 
     @Override
-    public void disableControllerService(final ControllerServiceNode serviceNode) {
+    public CompletableFuture<Void> disableControllerService(final ControllerServiceNode serviceNode) {
         serviceNode.verifyCanDisable();
-        controllerServiceProvider.disableControllerService(serviceNode);
+        return controllerServiceProvider.disableControllerService(serviceNode);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
index 01dd35e..58bb90f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
@@ -16,6 +16,34 @@
  */
 package org.apache.nifi.controller;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.zip.GZIPInputStream;
+
+import javax.xml.XMLConstants;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.validation.Schema;
+import javax.xml.validation.SchemaFactory;
+
 import org.apache.commons.collections4.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
@@ -92,33 +120,6 @@ import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 import org.xml.sax.SAXException;
 
-import javax.xml.XMLConstants;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URL;
-import java.nio.charset.StandardCharsets;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.StandardOpenOption;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
-import java.util.zip.GZIPInputStream;
-
 /**
  */
 public class StandardFlowSynchronizer implements FlowSynchronizer {
@@ -1037,6 +1038,21 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
             parentGroup.addProcessGroup(processGroup);
         }
 
+        // Set the variables for the variable registry
+        final Map<String, String> variables = new HashMap<>();
+        final List<Element> variableElements = getChildrenByTagName(processGroupElement, "variable");
+        for (final Element variableElement : variableElements) {
+            final String variableName = variableElement.getAttribute("name");
+            final String variableValue = variableElement.getAttribute("value");
+            if (variableName == null || variableValue == null) {
+                continue;
+            }
+
+            variables.put(variableName, variableValue);
+        }
+
+        processGroup.setVariables(variables);
+
         // Add Controller Services
         final List<Element> serviceNodeList = getChildrenByTagName(processGroupElement, "controllerService");
         if (!serviceNodeList.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
index 1ff09d7..36cb62e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
@@ -16,6 +16,31 @@
  */
 package org.apache.nifi.controller;
 
+import static java.util.Objects.requireNonNull;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
 import org.apache.commons.lang3.builder.EqualsBuilder;
 import org.apache.commons.lang3.builder.HashCodeBuilder;
 import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
@@ -56,7 +81,7 @@ import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.processor.Processor;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.SimpleProcessLogger;
-import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.ComponentVariableRegistry;
 import org.apache.nifi.scheduling.ExecutionNode;
 import org.apache.nifi.scheduling.SchedulingStrategy;
 import org.apache.nifi.util.CharacterFilterUtils;
@@ -68,30 +93,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.util.Assert;
 
-import java.lang.reflect.InvocationTargetException;
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static java.util.Objects.requireNonNull;
-
 /**
  * ProcessorNode provides thread-safe access to a FlowFileProcessor as it exists
  * within a controlled flow. This node keeps track of the processor, its
@@ -137,7 +138,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
     public StandardProcessorNode(final LoggableComponent<Processor> processor, final String uuid,
                                  final ValidationContextFactory validationContextFactory, final ProcessScheduler scheduler,
                                  final ControllerServiceProvider controllerServiceProvider, final NiFiProperties nifiProperties,
-                                 final VariableRegistry variableRegistry, final ReloadComponent reloadComponent) {
+                                 final ComponentVariableRegistry variableRegistry, final ReloadComponent reloadComponent) {
 
         this(processor, uuid, validationContextFactory, scheduler, controllerServiceProvider,
             processor.getComponent().getClass().getSimpleName(), processor.getComponent().getClass().getCanonicalName(), nifiProperties, variableRegistry, reloadComponent, false);
@@ -147,7 +148,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
                                  final ValidationContextFactory validationContextFactory, final ProcessScheduler scheduler,
                                  final ControllerServiceProvider controllerServiceProvider,
                                  final String componentType, final String componentCanonicalClass, final NiFiProperties nifiProperties,
-                                 final VariableRegistry variableRegistry, final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
+                                 final ComponentVariableRegistry variableRegistry, final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
 
         super(uuid, validationContextFactory, controllerServiceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent, isExtensionMissing);
 
@@ -1372,13 +1373,13 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
      * </p>
      */
     @Override
-    public <T extends ProcessContext & ControllerServiceLookup> void stop(final ScheduledExecutorService scheduler,
+    public <T extends ProcessContext & ControllerServiceLookup> CompletableFuture<Void> stop(final ScheduledExecutorService scheduler,
         final T processContext, final SchedulingAgent schedulingAgent, final ScheduleState scheduleState) {
 
         final Processor processor = processorRef.get().getProcessor();
         LOG.info("Stopping processor: " + processor.getClass());
 
-
+        final CompletableFuture<Void> future = new CompletableFuture<>();
         if (this.scheduledState.compareAndSet(ScheduledState.RUNNING, ScheduledState.STOPPING)) { // will ensure that the Processor represented by this node can only be stopped once
             scheduleState.incrementActiveThreadCount();
 
@@ -1405,6 +1406,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
 
                             scheduleState.decrementActiveThreadCount();
                             scheduledState.set(ScheduledState.STOPPED);
+                            future.complete(null);
                         } else {
                             // Not all of the active threads have finished. Try again in 100 milliseconds.
                             scheduler.schedule(this, 100, TimeUnit.MILLISECONDS);
@@ -1415,16 +1417,17 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
                 }
             });
         } else {
-        /*
-         * We do compareAndSet() instead of set() to ensure that Processor
-         * stoppage is handled consistently including a condition where
-         * Processor never got a chance to transition to RUNNING state
-         * before stop() was called. If that happens the stop processor
-         * routine will be initiated in start() method, otherwise the IF
-         * part will handle the stop processor routine.
-         */
+            // We do compareAndSet() instead of set() to ensure that Processor
+            // stoppage is handled consistently including a condition where
+            // Processor never got a chance to transition to RUNNING state
+            // before stop() was called. If that happens the stop processor
+            // routine will be initiated in start() method, otherwise the IF
+            // part will handle the stop processor routine.
             this.scheduledState.compareAndSet(ScheduledState.STARTING, ScheduledState.STOPPING);
+            future.complete(null);
         }
+
+        return future;
     }
 
     /**