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

[1/2] nifi git commit: NIFI-4997: - Fixing process group audit advice. - Setting spring security user in background threads. - Removing unnecessary overloaded methods.

Repository: nifi
Updated Branches:
  refs/heads/master 6fbe1515e -> b7272e3f3


http://git-wip-us.apache.org/repos/asf/nifi/blob/b7272e3f/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/ClusterReplicationComponentLifecycle.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/ClusterReplicationComponentLifecycle.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/ClusterReplicationComponentLifecycle.java
index 28fef0f..a2243b5 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/ClusterReplicationComponentLifecycle.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/ClusterReplicationComponentLifecycle.java
@@ -18,6 +18,7 @@
 package org.apache.nifi.web.util;
 
 import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.authorization.user.NiFiUserUtils;
 import org.apache.nifi.cluster.coordination.ClusterCoordinator;
 import org.apache.nifi.cluster.coordination.http.replication.RequestReplicator;
 import org.apache.nifi.cluster.exception.NoClusterCoordinatorException;
@@ -65,7 +66,7 @@ public class ClusterReplicationComponentLifecycle implements ComponentLifecycle
 
 
     @Override
-    public Set<AffectedComponentEntity> scheduleComponents(final URI exampleUri, final NiFiUser user, final String groupId, final Set<AffectedComponentEntity> components,
+    public Set<AffectedComponentEntity> scheduleComponents(final URI exampleUri, final String groupId, final Set<AffectedComponentEntity> components,
             final ScheduledState desiredState, final Pause pause) throws LifecycleManagementException {
 
         final Set<String> componentIds = components.stream()
@@ -95,6 +96,8 @@ public class ClusterReplicationComponentLifecycle implements ComponentLifecycle
         final Map<String, String> headers = new HashMap<>();
         headers.put("content-type", MediaType.APPLICATION_JSON);
 
+        final NiFiUser user = NiFiUserUtils.getNiFiUser();
+
         // Determine whether we should replicate only to the cluster coordinator, or if we should replicate directly to the cluster nodes themselves.
         try {
             final NodeResponse clusterResponse;
@@ -122,7 +125,7 @@ public class ClusterReplicationComponentLifecycle implements ComponentLifecycle
         }
 
         final Set<AffectedComponentEntity> updatedEntities = components.stream()
-            .map(component -> AffectedComponentUtils.updateEntity(component, serviceFacade, dtoFactory, user))
+            .map(component -> AffectedComponentUtils.updateEntity(component, serviceFacade, dtoFactory))
             .collect(Collectors.toSet());
         return updatedEntities;
     }
@@ -274,7 +277,7 @@ public class ClusterReplicationComponentLifecycle implements ComponentLifecycle
 
 
     @Override
-    public Set<AffectedComponentEntity> activateControllerServices(final URI originalUri, final NiFiUser user, final String groupId, final Set<AffectedComponentEntity> affectedServices,
+    public Set<AffectedComponentEntity> activateControllerServices(final URI originalUri, final String groupId, final Set<AffectedComponentEntity> affectedServices,
         final ControllerServiceState desiredState, final Pause pause) throws LifecycleManagementException {
 
         final Set<String> affectedServiceIds = affectedServices.stream()
@@ -301,6 +304,8 @@ public class ClusterReplicationComponentLifecycle implements ComponentLifecycle
         final Map<String, String> headers = new HashMap<>();
         headers.put("content-type", MediaType.APPLICATION_JSON);
 
+        final NiFiUser user = NiFiUserUtils.getNiFiUser();
+
         // Determine whether we should replicate only to the cluster coordinator, or if we should replicate directly to the cluster nodes themselves.
         try {
             final NodeResponse clusterResponse;
@@ -328,7 +333,7 @@ public class ClusterReplicationComponentLifecycle implements ComponentLifecycle
         }
 
         return affectedServices.stream()
-            .map(componentEntity -> serviceFacade.getControllerService(componentEntity.getId(), user))
+            .map(componentEntity -> serviceFacade.getControllerService(componentEntity.getId()))
             .map(dtoFactory::createAffectedComponentEntity)
             .collect(Collectors.toSet());
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/b7272e3f/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/ComponentLifecycle.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/ComponentLifecycle.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/ComponentLifecycle.java
index c84b966..687c370 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/ComponentLifecycle.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/ComponentLifecycle.java
@@ -17,20 +17,18 @@
 
 package org.apache.nifi.web.util;
 
-import java.net.URI;
-import java.util.Set;
-
-import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.service.ControllerServiceState;
 import org.apache.nifi.web.api.entity.AffectedComponentEntity;
 
+import java.net.URI;
+import java.util.Set;
+
 public interface ComponentLifecycle {
     /**
      * Updates the scheduled state of all components that are given, to match the desired ScheduledState
      *
      * @param exampleUri an URI to use as a base for the REST API.
-     * @param user the user making the request
      * @param groupId the ID of the process group
      * @param components the components to schedule or unschedule
      * @param desiredState the desired state of the components
@@ -40,14 +38,13 @@ public interface ComponentLifecycle {
      *
      * @throws IllegalStateException if any of the components given do not have a state that can be transitioned to the given desired state
      */
-    Set<AffectedComponentEntity> scheduleComponents(URI exampleUri, NiFiUser user, String groupId, Set<AffectedComponentEntity> components,
+    Set<AffectedComponentEntity> scheduleComponents(URI exampleUri, String groupId, Set<AffectedComponentEntity> components,
         ScheduledState desiredState, Pause pause) throws LifecycleManagementException;
 
     /**
      * Updates the Controller Service State state of all controller services that are given, to match the desired ControllerServiceState
      *
      * @param exampleUri an URI to use as a base for the REST API
-     * @param user the user making the request
      * @param groupId the ID of the process group
      * @param services the controller services to enable or disable
      * @param desiredState the desired state of the components
@@ -57,6 +54,6 @@ public interface ComponentLifecycle {
      *
      * @throws IllegalStateException if any of the components given do not have a state that can be transitioned to the given desired state
      */
-    Set<AffectedComponentEntity> activateControllerServices(URI exampleUri, NiFiUser user, String groupId, Set<AffectedComponentEntity> services,
+    Set<AffectedComponentEntity> activateControllerServices(URI exampleUri, String groupId, Set<AffectedComponentEntity> services,
         ControllerServiceState desiredState, Pause pause) throws LifecycleManagementException;
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/b7272e3f/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/LocalComponentLifecycle.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/LocalComponentLifecycle.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/LocalComponentLifecycle.java
index 1c7e82d..8a08684 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/LocalComponentLifecycle.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/LocalComponentLifecycle.java
@@ -17,7 +17,6 @@
 
 package org.apache.nifi.web.util;
 
-import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.service.ControllerServiceNode;
@@ -51,7 +50,7 @@ public class LocalComponentLifecycle implements ComponentLifecycle {
     private DtoFactory dtoFactory;
 
     @Override
-    public Set<AffectedComponentEntity> scheduleComponents(final URI exampleUri, final NiFiUser user, final String groupId, final Set<AffectedComponentEntity> components,
+    public Set<AffectedComponentEntity> scheduleComponents(final URI exampleUri, final String groupId, final Set<AffectedComponentEntity> components,
         final ScheduledState desiredState, final Pause pause) throws LifecycleManagementException {
 
         final Map<String, Revision> processorRevisions = components.stream()
@@ -61,19 +60,19 @@ public class LocalComponentLifecycle implements ComponentLifecycle {
             .collect(Collectors.toMap(AffectedComponentEntity::getId, Function.identity()));
 
         if (desiredState == ScheduledState.RUNNING) {
-            startComponents(groupId, processorRevisions, affectedComponentMap, user, pause);
+            startComponents(groupId, processorRevisions, affectedComponentMap, pause);
         } else {
-            stopComponents(groupId, processorRevisions, affectedComponentMap, user, pause);
+            stopComponents(groupId, processorRevisions, affectedComponentMap, pause);
         }
 
         final Set<AffectedComponentEntity> updatedEntities = components.stream()
-            .map(component -> AffectedComponentUtils.updateEntity(component, serviceFacade, dtoFactory, user))
+            .map(component -> AffectedComponentUtils.updateEntity(component, serviceFacade, dtoFactory))
             .collect(Collectors.toSet());
         return updatedEntities;
     }
 
     @Override
-    public Set<AffectedComponentEntity> activateControllerServices(final URI exampleUri, final NiFiUser user, final String groupId, final Set<AffectedComponentEntity> services,
+    public Set<AffectedComponentEntity> activateControllerServices(final URI exampleUri, final String groupId, final Set<AffectedComponentEntity> services,
         final ControllerServiceState desiredState, final Pause pause) throws LifecycleManagementException {
 
         final Map<String, Revision> serviceRevisions = services.stream()
@@ -83,20 +82,19 @@ public class LocalComponentLifecycle implements ComponentLifecycle {
             .collect(Collectors.toMap(AffectedComponentEntity::getId, Function.identity()));
 
         if (desiredState == ControllerServiceState.ENABLED) {
-            enableControllerServices(groupId, serviceRevisions, affectedServiceMap, user, pause);
+            enableControllerServices(groupId, serviceRevisions, affectedServiceMap, pause);
         } else {
-            disableControllerServices(groupId, serviceRevisions, affectedServiceMap, user, pause);
+            disableControllerServices(groupId, serviceRevisions, affectedServiceMap, pause);
         }
 
         return services.stream()
-            .map(componentEntity -> serviceFacade.getControllerService(componentEntity.getId(), user))
+            .map(componentEntity -> serviceFacade.getControllerService(componentEntity.getId()))
             .map(dtoFactory::createAffectedComponentEntity)
             .collect(Collectors.toSet());
     }
 
 
-    private void startComponents(final String processGroupId, final Map<String, Revision> componentRevisions, final Map<String, AffectedComponentEntity> affectedComponents,
-        final NiFiUser user, final Pause pause) {
+    private void startComponents(final String processGroupId, final Map<String, Revision> componentRevisions, final Map<String, AffectedComponentEntity> affectedComponents, final Pause pause) {
 
         if (componentRevisions.isEmpty()) {
             return;
@@ -105,15 +103,14 @@ public class LocalComponentLifecycle implements ComponentLifecycle {
         logger.debug("Starting components with ID's {} from Process Group {}", componentRevisions.keySet(), processGroupId);
 
         serviceFacade.verifyScheduleComponents(processGroupId, ScheduledState.RUNNING, componentRevisions.keySet());
-        serviceFacade.scheduleComponents(user, processGroupId, ScheduledState.RUNNING, componentRevisions);
+        serviceFacade.scheduleComponents(processGroupId, ScheduledState.RUNNING, componentRevisions);
 
         // wait for all of the Processors to reach the desired state. We don't have to wait for other components because
         // Local and Remote Ports as well as funnels start immediately.
         waitForProcessorState(processGroupId, affectedComponents, ScheduledState.RUNNING, pause);
     }
 
-    private void stopComponents(final String processGroupId, final Map<String, Revision> componentRevisions, final Map<String, AffectedComponentEntity> affectedComponents,
-        final NiFiUser user, final Pause pause) {
+    private void stopComponents(final String processGroupId, final Map<String, Revision> componentRevisions, final Map<String, AffectedComponentEntity> affectedComponents, final Pause pause) {
 
         if (componentRevisions.isEmpty()) {
             return;
@@ -122,7 +119,7 @@ public class LocalComponentLifecycle implements ComponentLifecycle {
         logger.debug("Stopping components with ID's {} from Process Group {}", componentRevisions.keySet(), processGroupId);
 
         serviceFacade.verifyScheduleComponents(processGroupId, ScheduledState.STOPPED, componentRevisions.keySet());
-        serviceFacade.scheduleComponents(user, processGroupId, ScheduledState.STOPPED, componentRevisions);
+        serviceFacade.scheduleComponents(processGroupId, ScheduledState.STOPPED, componentRevisions);
 
         // wait for all of the Processors to reach the desired state. We don't have to wait for other components because
         // Local and Remote Ports as well as funnels stop immediately.
@@ -205,8 +202,7 @@ public class LocalComponentLifecycle implements ComponentLifecycle {
     }
 
 
-    private void enableControllerServices(final String processGroupId, final Map<String, Revision> serviceRevisions, final Map<String, AffectedComponentEntity> affectedServices,
-        final NiFiUser user, final Pause pause) {
+    private void enableControllerServices(final String processGroupId, final Map<String, Revision> serviceRevisions, final Map<String, AffectedComponentEntity> affectedServices, final Pause pause) {
 
         if (serviceRevisions.isEmpty()) {
             return;
@@ -215,12 +211,11 @@ public class LocalComponentLifecycle implements ComponentLifecycle {
         logger.debug("Enabling Controller Services with ID's {} from Process Group {}", serviceRevisions.keySet(), processGroupId);
 
         serviceFacade.verifyActivateControllerServices(processGroupId, ControllerServiceState.ENABLED, affectedServices.keySet());
-        serviceFacade.activateControllerServices(user, processGroupId, ControllerServiceState.ENABLED, serviceRevisions);
-        waitForControllerServiceState(processGroupId, affectedServices, ControllerServiceState.ENABLED, pause, user);
+        serviceFacade.activateControllerServices(processGroupId, ControllerServiceState.ENABLED, serviceRevisions);
+        waitForControllerServiceState(processGroupId, affectedServices, ControllerServiceState.ENABLED, pause);
     }
 
-    private void disableControllerServices(final String processGroupId, final Map<String, Revision> serviceRevisions, final Map<String, AffectedComponentEntity> affectedServices,
-        final NiFiUser user, final Pause pause) {
+    private void disableControllerServices(final String processGroupId, final Map<String, Revision> serviceRevisions, final Map<String, AffectedComponentEntity> affectedServices, final Pause pause) {
 
         if (serviceRevisions.isEmpty()) {
             return;
@@ -229,8 +224,8 @@ public class LocalComponentLifecycle implements ComponentLifecycle {
         logger.debug("Disabling Controller Services with ID's {} from Process Group {}", serviceRevisions.keySet(), processGroupId);
 
         serviceFacade.verifyActivateControllerServices(processGroupId, ControllerServiceState.DISABLED, affectedServices.keySet());
-        serviceFacade.activateControllerServices(user, processGroupId, ControllerServiceState.DISABLED, serviceRevisions);
-        waitForControllerServiceState(processGroupId, affectedServices, ControllerServiceState.DISABLED, pause, user);
+        serviceFacade.activateControllerServices(processGroupId, ControllerServiceState.DISABLED, serviceRevisions);
+        waitForControllerServiceState(processGroupId, affectedServices, ControllerServiceState.DISABLED, pause);
     }
 
     static List<List<ControllerServiceNode>> determineEnablingOrder(final Map<String, ControllerServiceNode> serviceNodeMap) {
@@ -280,17 +275,15 @@ public class LocalComponentLifecycle implements ComponentLifecycle {
      * @param affectedServices 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 waitForControllerServiceState(final String groupId, final Map<String, AffectedComponentEntity> affectedServices, final ControllerServiceState desiredState, final Pause pause,
-        final NiFiUser user) {
+    private boolean waitForControllerServiceState(final String groupId, final Map<String, AffectedComponentEntity> affectedServices, final ControllerServiceState desiredState, final Pause pause) {
 
         logger.debug("Waiting for {} Controller Services to transition their states to {}", affectedServices.size(), desiredState);
 
         boolean continuePolling = true;
         while (continuePolling) {
-            final Set<ControllerServiceEntity> serviceEntities = serviceFacade.getControllerServices(groupId, false, true, user);
+            final Set<ControllerServiceEntity> serviceEntities = serviceFacade.getControllerServices(groupId, false, true);
 
             // update the affected controller services
             updateAffectedControllerServices(serviceEntities, affectedServices);


[2/2] nifi git commit: NIFI-4997: - Fixing process group audit advice. - Setting spring security user in background threads. - Removing unnecessary overloaded methods.

Posted by ma...@apache.org.
NIFI-4997:
- Fixing process group audit advice.
- Setting spring security user in background threads.
- Removing unnecessary overloaded methods.

This closes #2626.

Signed-off-by: Mark Payne <ma...@hotmail.com>


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

Branch: refs/heads/master
Commit: b7272e3f3282c6e42eb7ada86d15f32188527ccf
Parents: 6fbe151
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Apr 10 10:17:49 2018 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Wed Apr 11 16:18:36 2018 -0400

----------------------------------------------------------------------
 .../apache/nifi/audit/ProcessGroupAuditor.java  |  42 +++---
 .../org/apache/nifi/web/NiFiServiceFacade.java  | 102 +------------
 .../nifi/web/StandardNiFiServiceFacade.java     | 149 ++++++-------------
 .../nifi/web/api/ProcessGroupResource.java      |  46 +++---
 .../apache/nifi/web/api/VersionsResource.java   |  33 ++--
 .../web/api/concurrent/AsyncRequestManager.java |  21 ++-
 .../org/apache/nifi/web/api/dto/DtoFactory.java |  12 +-
 .../apache/nifi/web/dao/ProcessGroupDAO.java    |   9 +-
 .../web/dao/impl/StandardProcessGroupDAO.java   |   7 +-
 .../nifi/web/util/AffectedComponentUtils.java   |  16 +-
 .../ClusterReplicationComponentLifecycle.java   |  13 +-
 .../nifi/web/util/ComponentLifecycle.java       |  13 +-
 .../nifi/web/util/LocalComponentLifecycle.java  |  47 +++---
 13 files changed, 174 insertions(+), 336 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/b7272e3f/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 363e049..3ba5bea 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
@@ -42,6 +42,7 @@ import org.slf4j.LoggerFactory;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Date;
+import java.util.Set;
 import java.util.concurrent.Future;
 
 /**
@@ -63,7 +64,7 @@ public class ProcessGroupAuditor extends NiFiAuditor {
      * @throws java.lang.Throwable ex
      */
     @Around("within(org.apache.nifi.web.dao.ProcessGroupDAO+) && "
-            + "execution(org.apache.nifi.groups.ProcessGroup createProcessGroup(java.lang.String, org.apache.nifi.web.api.dto.ProcessGroupDTO))")
+            + "execution(org.apache.nifi.groups.ProcessGroup createProcessGroup(String, org.apache.nifi.web.api.dto.ProcessGroupDTO))")
     public ProcessGroup createProcessGroupAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable {
         // create the process group
         ProcessGroup processGroup = (ProcessGroup) proceedingJoinPoint.proceed();
@@ -177,9 +178,9 @@ public class ProcessGroupAuditor extends NiFiAuditor {
      * @throws Throwable ex
      */
     @Around("within(org.apache.nifi.web.dao.ProcessGroupDAO+) && "
-        + "execution(java.util.concurrent.Future<Void> scheduleComponents(java.lang.String, org.apache.nifi.controller.ScheduledState, java.util.Set)) && "
-        + "args(groupId, state)")
-    public Future<Void> scheduleComponentsAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, ScheduledState state) throws Throwable {
+        + "execution(java.util.concurrent.Future<Void> scheduleComponents(String, org.apache.nifi.controller.ScheduledState, java.util.Set<String>)) && "
+        + "args(groupId, state, componentIds)")
+    public Future<Void> scheduleComponentsAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, ScheduledState state, Set<String> componentIds) throws Throwable {
         final Operation operation;
 
         final Future<Void> result = (Future<Void>) proceedingJoinPoint.proceed();
@@ -191,7 +192,7 @@ public class ProcessGroupAuditor extends NiFiAuditor {
             operation = Operation.Stop;
         }
 
-        saveUpdateAction(NiFiUserUtils.getNiFiUser(), groupId, operation);
+        saveUpdateAction(groupId, operation);
 
         return result;
     }
@@ -206,9 +207,9 @@ public class ProcessGroupAuditor extends NiFiAuditor {
      * @throws Throwable ex
      */
     @Around("within(org.apache.nifi.web.dao.ProcessGroupDAO+) && "
-        + "execution(java.util.concurrent.Future<Void> activateControllerServices(java.lang.String, org.apache.nifi.controller.service.ControllerServiceState, java.util.Set)) && "
-        + "args(groupId, state)")
-    public Future<Void> activateControllerServicesAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, ControllerServiceState state) throws Throwable {
+        + "execution(java.util.concurrent.Future<Void> activateControllerServices(String, org.apache.nifi.controller.service.ControllerServiceState, java.util.Collection<String>)) && "
+        + "args(groupId, state, serviceIds)")
+    public Future<Void> activateControllerServicesAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, ControllerServiceState state, Collection<String> serviceIds) throws Throwable {
         final Operation operation;
 
         final Future<Void> result = (Future<Void>) proceedingJoinPoint.proceed();
@@ -220,7 +221,7 @@ public class ProcessGroupAuditor extends NiFiAuditor {
             operation = Operation.Disable;
         }
 
-        saveUpdateAction(NiFiUserUtils.getNiFiUser(), groupId, operation);
+        saveUpdateAction(groupId, operation);
 
         return result;
     }
@@ -229,17 +230,16 @@ public class ProcessGroupAuditor extends NiFiAuditor {
      * Audits the update of process group variable registry.
      *
      * @param proceedingJoinPoint join point
-     * @param user the user performing the action
      * @param variableRegistry variable registry
      * @throws Throwable ex
      */
     @Around("within(org.apache.nifi.web.dao.ProcessGroupDAO+) && "
-        + "execution(org.apache.nifi.groups.ProcessGroup updateVariableRegistry(org.apache.nifi.authorization.user.NiFiUser, org.apache.nifi.web.api.dto.VariableRegistryDTO)) && "
-        + "args(user, variableRegistry)")
-    public ProcessGroup updateVariableRegistryAdvice(final ProceedingJoinPoint proceedingJoinPoint, final NiFiUser user, final VariableRegistryDTO variableRegistry) throws Throwable {
+        + "execution(org.apache.nifi.groups.ProcessGroup updateVariableRegistry(org.apache.nifi.web.api.dto.VariableRegistryDTO)) && "
+        + "args(variableRegistry)")
+    public ProcessGroup updateVariableRegistryAdvice(final ProceedingJoinPoint proceedingJoinPoint, final VariableRegistryDTO variableRegistry) throws Throwable {
         final ProcessGroup updatedProcessGroup = (ProcessGroup) proceedingJoinPoint.proceed();
 
-        saveUpdateAction(user, variableRegistry.getProcessGroupId(), Operation.Configure);
+        saveUpdateAction(variableRegistry.getProcessGroupId(), Operation.Configure);
 
         return updatedProcessGroup;
     }
@@ -249,7 +249,6 @@ public class ProcessGroupAuditor extends NiFiAuditor {
     public ProcessGroup updateProcessGroupFlowAdvice(final ProceedingJoinPoint proceedingJoinPoint) throws Throwable {
         final Object[] args = proceedingJoinPoint.getArgs();
         final String groupId = (String) args[0];
-        final NiFiUser user = (NiFiUser) args[1];
 
         final ProcessGroupDAO processGroupDAO = getProcessGroupDAO();
         final ProcessGroup processGroup = processGroupDAO.getProcessGroup(groupId);
@@ -271,7 +270,7 @@ public class ProcessGroupAuditor extends NiFiAuditor {
             }
         }
 
-        saveUpdateAction(user, groupId, operation);
+        saveUpdateAction(groupId, operation);
 
         return updatedProcessGroup;
     }
@@ -294,23 +293,24 @@ public class ProcessGroupAuditor extends NiFiAuditor {
             operation = Operation.CommitLocalChanges;
         }
 
-        saveUpdateAction(NiFiUserUtils.getNiFiUser(), vciDto.getGroupId(), operation);
+        saveUpdateAction(vciDto.getGroupId(), operation);
 
         return updatedProcessGroup;
     }
 
     @Around("within(org.apache.nifi.web.dao.ProcessGroupDAO+) && "
-            + "execution(org.apache.nifi.groups.ProcessGroup disconnectVersionControl(java.lang.String)) && "
+            + "execution(org.apache.nifi.groups.ProcessGroup disconnectVersionControl(String)) && "
             + "args(groupId)")
     public ProcessGroup disconnectVersionControlAdvice(final ProceedingJoinPoint proceedingJoinPoint, final String groupId) throws Throwable {
         final ProcessGroup updatedProcessGroup = (ProcessGroup) proceedingJoinPoint.proceed();
 
-        saveUpdateAction(NiFiUserUtils.getNiFiUser(), groupId, Operation.StopVersionControl);
+        saveUpdateAction(groupId, Operation.StopVersionControl);
 
         return updatedProcessGroup;
     }
 
-    private void saveUpdateAction(final NiFiUser user, final String groupId, final Operation operation) throws Throwable {
+    private void saveUpdateAction(final String groupId, final Operation operation) throws Throwable {
+        NiFiUser user = NiFiUserUtils.getNiFiUser();
         ProcessGroupDAO processGroupDAO = getProcessGroupDAO();
         ProcessGroup processGroup = processGroupDAO.getProcessGroup(groupId);
 
@@ -335,7 +335,7 @@ public class ProcessGroupAuditor extends NiFiAuditor {
      * @throws Throwable ex
      */
     @Around("within(org.apache.nifi.web.dao.ProcessGroupDAO+) && "
-            + "execution(void deleteProcessGroup(java.lang.String)) && "
+            + "execution(void deleteProcessGroup(String)) && "
             + "args(groupId)")
     public void removeProcessGroupAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId) throws Throwable {
         // get the process group before removing it

http://git-wip-us.apache.org/repos/asf/nifi/blob/b7272e3f/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 0e3d061..ba813ff 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
@@ -549,14 +549,6 @@ public interface NiFiServiceFacade {
     ProcessorDiagnosticsEntity getProcessorDiagnostics(String id);
 
     /**
-     * Gets the Processor transfer object for the specified id, as it is visible to the given user
-     *
-     * @param id Id of the processor to return
-     * @return The Processor transfer object
-     */
-    ProcessorEntity getProcessor(String id, NiFiUser user);
-
-    /**
      * Gets the processor status.
      *
      * @param id id
@@ -803,15 +795,6 @@ public interface NiFiServiceFacade {
     PortEntity getInputPort(String inputPortId);
 
     /**
-     * Gets an input port as it is available to the given user
-     *
-     * @param inputPortId The input port id
-     * @param user the user
-     * @return port
-     */
-    PortEntity getInputPort(String inputPortId, NiFiUser user);
-
-    /**
      * Gets all input ports in a given group.
      *
      * @param groupId The id of the group
@@ -881,15 +864,6 @@ public interface NiFiServiceFacade {
     PortEntity getOutputPort(String outputPortId);
 
     /**
-     * Gets an output port as it is available to the given user
-     *
-     * @param outputPortId The output port id
-     * @param user the user
-     * @return port
-     */
-    PortEntity getOutputPort(String outputPortId, NiFiUser user);
-
-    /**
      * Gets all output ports in a given group.
      *
      * @param groupId The id of the group
@@ -1002,15 +976,6 @@ public interface NiFiServiceFacade {
     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
@@ -1064,17 +1029,6 @@ public interface NiFiServiceFacade {
     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
@@ -1085,17 +1039,6 @@ 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
@@ -1162,16 +1105,6 @@ public interface NiFiServiceFacade {
     RemoteProcessGroupEntity getRemoteProcessGroup(String remoteProcessGroupId);
 
     /**
-     * Gets a remote process group as it is visible to the given user
-     *
-     * @param remoteProcessGroupId The id of the remote process group
-     * @param user the user requesting the action
-     * @return group
-     */
-    RemoteProcessGroupEntity getRemoteProcessGroup(String remoteProcessGroupId, NiFiUser user);
-
-
-    /**
      * Gets all remote process groups in the a given parent group.
      *
      * @param groupId The id of the parent group
@@ -1180,15 +1113,6 @@ public interface NiFiServiceFacade {
     Set<RemoteProcessGroupEntity> getRemoteProcessGroups(String groupId);
 
     /**
-     * Gets all remote process groups in the a given parent group as they are visible to the given user
-     *
-     * @param groupId The id of the parent group
-     * @param user the user making the request
-     * @return group
-     */
-    Set<RemoteProcessGroupEntity> getRemoteProcessGroups(String groupId, NiFiUser user);
-
-    /**
      * Gets the remote process group status.
      *
      * @param id remote process group
@@ -1456,10 +1380,9 @@ public interface NiFiServiceFacade {
      *
      * @param processGroupId the ID of the Process Group to update
      * @param updatedSnapshot the snapshot to update the Process Group to
-     * @param user the user making the request
      * @return the set of all components that would be affected by updating the Process Group
      */
-    Set<AffectedComponentEntity> getComponentsAffectedByVersionChange(String processGroupId, VersionedFlowSnapshot updatedSnapshot, NiFiUser user);
+    Set<AffectedComponentEntity> getComponentsAffectedByVersionChange(String processGroupId, VersionedFlowSnapshot updatedSnapshot);
 
     /**
      * Verifies that the Process Group with the given identifier can be updated to the proposed flow
@@ -1499,7 +1422,6 @@ public interface NiFiServiceFacade {
     /**
      * Updates the Process group with the given ID to match the new snapshot
      *
-     * @param user the user performing the request
      * @param revision the revision of the Process Group
      * @param groupId the ID of the Process Group
      * @param versionControlInfo the Version Control information
@@ -1510,7 +1432,7 @@ public interface NiFiServiceFacade {
      *            update the contents of that Process Group
      * @return the Process Group
      */
-    ProcessGroupEntity updateProcessGroupContents(NiFiUser user, Revision revision, String groupId, VersionControlInformationDTO versionControlInfo, VersionedFlowSnapshot snapshot,
+    ProcessGroupEntity updateProcessGroupContents(Revision revision, String groupId, VersionControlInformationDTO versionControlInfo, VersionedFlowSnapshot snapshot,
                                                   String componentIdSeed, boolean verifyNotModified, boolean updateSettings, boolean updateDescendantVersionedFlows);
 
     // ----------------------------------------
@@ -1785,17 +1707,6 @@ public interface NiFiServiceFacade {
     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, boolean includeAncestorGroups, boolean includeDescendantGroups, NiFiUser user);
-
-    /**
      * Gets the specified controller service.
      *
      * @param controllerServiceId id
@@ -1804,15 +1715,6 @@ public interface NiFiServiceFacade {
     ControllerServiceEntity getControllerService(String controllerServiceId);
 
     /**
-     * Gets the specified controller service as it is visible to the given user
-     *
-     * @param controllerServiceId id
-     * @param user the user making the request
-     * @return service
-     */
-    ControllerServiceEntity getControllerService(String controllerServiceId, NiFiUser user);
-
-    /**
      * Get the descriptor for the specified property of the specified controller service.
      *
      * @param id id

http://git-wip-us.apache.org/repos/asf/nifi/blob/b7272e3f/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 33cd36f..ff52883 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
@@ -701,12 +701,9 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
      * @return A RevisionUpdate that represents the new configuration
      */
     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 NiFiUser user = NiFiUserUtils.getNiFiUser();
+
             final RevisionUpdate<D> updatedComponent = revisionManager.updateRevision(new StandardRevisionClaim(revision), user, new UpdateRevisionTask<D>() {
                 @Override
                 public RevisionUpdate<D> update() {
@@ -936,18 +933,13 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
 
     @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,
+        final RevisionUpdate<VariableRegistryDTO> snapshot = updateComponent(revision,
             processGroupNode,
-            () -> processGroupDAO.updateVariableRegistry(user, variableRegistryDto),
-            processGroup -> dtoFactory.createVariableRegistryDto(processGroup, revisionManager, user));
+            () -> processGroupDAO.updateVariableRegistry(variableRegistryDto),
+            processGroup -> dtoFactory.createVariableRegistryDto(processGroup, revisionManager));
 
-        final PermissionsDTO permissions = dtoFactory.createPermissionsDto(processGroupNode, user);
+        final PermissionsDTO permissions = dtoFactory.createPermissionsDto(processGroupNode);
         final RevisionDTO updatedRevision = dtoFactory.createRevisionDTO(snapshot.getLastModification());
         return entityFactory.createVariableRegistryEntity(snapshot.getComponent(), updatedRevision, permissions);
     }
@@ -979,12 +971,6 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
     @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
@@ -1016,19 +1002,12 @@ 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(state, serviceRevisions.keySet());
+                    processGroupDAO.activateControllerServices(processGroupId, state, serviceRevisions.keySet());
 
                     // update the revisions
                     final Map<String, Revision> updatedRevisions = new HashMap<>();
@@ -2891,13 +2870,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
 
     @Override
     public ProcessorEntity getProcessor(final String id) {
-        return getProcessor(id, NiFiUserUtils.getNiFiUser());
-    }
-
-    @Override
-    public ProcessorEntity getProcessor(final String id, final NiFiUser user) {
         final ProcessorNode processor = processorDAO.getProcessor(id);
-        return createProcessorEntity(processor, user);
+        return createProcessorEntity(processor, NiFiUserUtils.getNiFiUser());
     }
 
     @Override
@@ -3329,12 +3303,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
     }
 
     private PortEntity createInputPortEntity(final Port port) {
-        return createInputPortEntity(port, NiFiUserUtils.getNiFiUser());
-    }
-
-    private PortEntity createInputPortEntity(final Port port, final NiFiUser user) {
         final RevisionDTO revision = dtoFactory.createRevisionDTO(revisionManager.getRevision(port.getIdentifier()));
-        final PermissionsDTO permissions = dtoFactory.createPermissionsDto(port, user);
+        final PermissionsDTO permissions = dtoFactory.createPermissionsDto(port, NiFiUserUtils.getNiFiUser());
         final PortStatusDTO status = dtoFactory.createPortStatusDto(controllerFacade.getInputPortStatus(port.getIdentifier()));
         final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(port.getIdentifier()));
         final List<BulletinEntity> bulletinEntities = bulletins.stream().map(bulletin -> entityFactory.createBulletinEntity(bulletin, permissions.getCanRead())).collect(Collectors.toList());
@@ -3342,12 +3312,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
     }
 
     private PortEntity createOutputPortEntity(final Port port) {
-        return createOutputPortEntity(port, NiFiUserUtils.getNiFiUser());
-    }
-
-    private PortEntity createOutputPortEntity(final Port port, final NiFiUser user) {
         final RevisionDTO revision = dtoFactory.createRevisionDTO(revisionManager.getRevision(port.getIdentifier()));
-        final PermissionsDTO permissions = dtoFactory.createPermissionsDto(port, user);
+        final PermissionsDTO permissions = dtoFactory.createPermissionsDto(port, NiFiUserUtils.getNiFiUser());
         final PortStatusDTO status = dtoFactory.createPortStatusDto(controllerFacade.getOutputPortStatus(port.getIdentifier()));
         final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(port.getIdentifier()));
         final List<BulletinEntity> bulletinEntities = bulletins.stream().map(bulletin -> entityFactory.createBulletinEntity(bulletin, permissions.getCanRead())).collect(Collectors.toList());
@@ -3439,11 +3405,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
 
     @Override
     public Set<RemoteProcessGroupEntity> getRemoteProcessGroups(final String groupId) {
-        return getRemoteProcessGroups(groupId, NiFiUserUtils.getNiFiUser());
-    }
-
-    @Override
-    public Set<RemoteProcessGroupEntity> getRemoteProcessGroups(final String groupId, final NiFiUser user) {
+        final NiFiUser user = NiFiUserUtils.getNiFiUser();
         final Set<RemoteProcessGroup> rpgs = remoteProcessGroupDAO.getRemoteProcessGroups(groupId);
         return rpgs.stream()
             .map(rpg -> createRemoteGroupEntity(rpg, user))
@@ -3457,12 +3419,6 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
     }
 
     @Override
-    public PortEntity getInputPort(final String inputPortId, final NiFiUser user) {
-        final Port port = inputPortDAO.getPort(inputPortId);
-        return createInputPortEntity(port, user);
-    }
-
-    @Override
     public PortStatusEntity getInputPortStatus(final String inputPortId) {
         final Port inputPort = inputPortDAO.getPort(inputPortId);
         final PermissionsDTO permissions = dtoFactory.createPermissionsDto(inputPort);
@@ -3477,12 +3433,6 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
     }
 
     @Override
-    public PortEntity getOutputPort(final String outputPortId, final NiFiUser user) {
-        final Port port = outputPortDAO.getPort(outputPortId);
-        return createOutputPortEntity(port, user);
-    }
-
-    @Override
     public PortStatusEntity getOutputPortStatus(final String outputPortId) {
         final Port outputPort = outputPortDAO.getPort(outputPortId);
         final PermissionsDTO permissions = dtoFactory.createPermissionsDto(outputPort);
@@ -3492,13 +3442,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
 
     @Override
     public RemoteProcessGroupEntity getRemoteProcessGroup(final String remoteProcessGroupId) {
-        return getRemoteProcessGroup(remoteProcessGroupId, NiFiUserUtils.getNiFiUser());
-    }
-
-    @Override
-    public RemoteProcessGroupEntity getRemoteProcessGroup(final String remoteProcessGroupId, final NiFiUser user) {
         final RemoteProcessGroup rpg = remoteProcessGroupDAO.getRemoteProcessGroup(remoteProcessGroupId);
-        return createRemoteGroupEntity(rpg, user);
+        return createRemoteGroupEntity(rpg, NiFiUserUtils.getNiFiUser());
     }
 
     @Override
@@ -3596,7 +3541,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
         return createProcessGroupEntity(processGroup);
     }
 
-    private ControllerServiceEntity createControllerServiceEntity(final ControllerServiceNode serviceNode, final Set<String> serviceIds, final NiFiUser user) {
+    private ControllerServiceEntity createControllerServiceEntity(final ControllerServiceNode serviceNode, final Set<String> serviceIds) {
         final ControllerServiceDTO dto = dtoFactory.createControllerServiceDto(serviceNode);
 
         final ControllerServiceReference ref = serviceNode.getReferences();
@@ -3604,7 +3549,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
         dto.setReferencingComponents(referencingComponentsEntity.getControllerServiceReferencingComponents());
 
         final RevisionDTO revision = dtoFactory.createRevisionDTO(revisionManager.getRevision(serviceNode.getIdentifier()));
-        final PermissionsDTO permissions = dtoFactory.createPermissionsDto(serviceNode, user);
+        final PermissionsDTO permissions = dtoFactory.createPermissionsDto(serviceNode, NiFiUserUtils.getNiFiUser());
         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);
@@ -3658,28 +3603,18 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
 
     @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, user))
+            .map(serviceNode -> createControllerServiceEntity(serviceNode, serviceIds))
             .collect(Collectors.toSet());
     }
 
     @Override
     public ControllerServiceEntity getControllerService(final String controllerServiceId) {
-        return getControllerService(controllerServiceId, NiFiUserUtils.getNiFiUser());
-    }
-
-    @Override
-    public ControllerServiceEntity getControllerService(final String controllerServiceId, final NiFiUser user) {
         final ControllerServiceNode controllerService = controllerServiceDAO.getControllerService(controllerServiceId);
-        return createControllerServiceEntity(controllerService, Sets.newHashSet(controllerServiceId), user);
+        return createControllerServiceEntity(controllerService, Sets.newHashSet(controllerServiceId));
     }
 
     @Override
@@ -4006,7 +3941,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
     }
 
     @Override
-    public Set<AffectedComponentEntity> getComponentsAffectedByVersionChange(final String processGroupId, final VersionedFlowSnapshot updatedSnapshot, final NiFiUser user) {
+    public Set<AffectedComponentEntity> getComponentsAffectedByVersionChange(final String processGroupId, final VersionedFlowSnapshot updatedSnapshot) {
         final ProcessGroup group = processGroupDAO.getProcessGroup(processGroupId);
 
         final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper();
@@ -4049,7 +3984,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
                         break;
                 }
 
-                return createAffectedComponentEntity((InstantiatedVersionedComponent) localComponent, localComponent.getComponentType().name(), state, user);
+                return createAffectedComponentEntity((InstantiatedVersionedComponent) localComponent, localComponent.getComponentType().name(), state);
             })
             .collect(Collectors.toCollection(HashSet::new));
 
@@ -4075,23 +4010,23 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
                 final ProcessGroup localGroup = processGroupDAO.getProcessGroup(localGroupId);
 
                 localGroup.findAllProcessors().stream()
-                    .map(comp -> createAffectedComponentEntity(comp, user))
+                    .map(comp -> createAffectedComponentEntity(comp))
                     .forEach(affectedComponents::add);
                 localGroup.findAllFunnels().stream()
-                    .map(comp -> createAffectedComponentEntity(comp, user))
+                    .map(comp -> createAffectedComponentEntity(comp))
                     .forEach(affectedComponents::add);
                 localGroup.findAllInputPorts().stream()
-                    .map(comp -> createAffectedComponentEntity(comp, user))
+                    .map(comp -> createAffectedComponentEntity(comp))
                     .forEach(affectedComponents::add);
                 localGroup.findAllOutputPorts().stream()
-                    .map(comp -> createAffectedComponentEntity(comp, user))
+                    .map(comp -> createAffectedComponentEntity(comp))
                     .forEach(affectedComponents::add);
                 localGroup.findAllRemoteProcessGroups().stream()
                     .flatMap(rpg -> Stream.concat(rpg.getInputPorts().stream(), rpg.getOutputPorts().stream()))
-                    .map(comp -> createAffectedComponentEntity(comp, user))
+                    .map(comp -> createAffectedComponentEntity(comp))
                     .forEach(affectedComponents::add);
                 localGroup.findAllControllerServices().stream()
-                    .map(comp -> createAffectedComponentEntity(comp, user))
+                    .map(comp -> createAffectedComponentEntity(comp))
                     .forEach(affectedComponents::add);
             }
 
@@ -4101,12 +4036,12 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
 
                 final List<ControllerServiceNode> referencingServices = serviceNode.getReferences().findRecursiveReferences(ControllerServiceNode.class);
                 for (final ControllerServiceNode referencingService : referencingServices) {
-                    affectedComponents.add(createAffectedComponentEntity(referencingService, user));
+                    affectedComponents.add(createAffectedComponentEntity(referencingService));
                 }
 
                 final List<ProcessorNode> referencingProcessors = serviceNode.getReferences().findRecursiveReferences(ProcessorNode.class);
                 for (final ProcessorNode referencingProcessor : referencingProcessors) {
-                    affectedComponents.add(createAffectedComponentEntity(referencingProcessor, user));
+                    affectedComponents.add(createAffectedComponentEntity(referencingProcessor));
                 }
             }
         }
@@ -4143,7 +4078,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
             final List<Connectable> sources = connectablesByVersionId.get(sourceVersionedId);
             if (sources != null) {
                 for (final Connectable source : sources) {
-                    affectedComponents.add(createAffectedComponentEntity(source, user));
+                    affectedComponents.add(createAffectedComponentEntity(source));
                 }
             }
 
@@ -4151,7 +4086,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
             final List<Connectable> destinations = connectablesByVersionId.get(destinationVersionId);
             if (destinations != null) {
                 for (final Connectable destination : destinations) {
-                    affectedComponents.add(createAffectedComponentEntity(destination, user));
+                    affectedComponents.add(createAffectedComponentEntity(destination));
                 }
             }
         }
@@ -4180,13 +4115,13 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
     }
 
 
-    private AffectedComponentEntity createAffectedComponentEntity(final Connectable connectable, final NiFiUser user) {
+    private AffectedComponentEntity createAffectedComponentEntity(final Connectable connectable) {
         final AffectedComponentEntity entity = new AffectedComponentEntity();
         entity.setRevision(dtoFactory.createRevisionDTO(revisionManager.getRevision(connectable.getIdentifier())));
         entity.setId(connectable.getIdentifier());
 
         final Authorizable authorizable = getAuthorizable(connectable);
-        final PermissionsDTO permissionsDto = dtoFactory.createPermissionsDto(authorizable, user);
+        final PermissionsDTO permissionsDto = dtoFactory.createPermissionsDto(authorizable);
         entity.setPermissions(permissionsDto);
 
         final AffectedComponentDTO dto = new AffectedComponentDTO();
@@ -4201,13 +4136,13 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
         return entity;
     }
 
-    private AffectedComponentEntity createAffectedComponentEntity(final ControllerServiceNode serviceNode, final NiFiUser user) {
+    private AffectedComponentEntity createAffectedComponentEntity(final ControllerServiceNode serviceNode) {
         final AffectedComponentEntity entity = new AffectedComponentEntity();
         entity.setRevision(dtoFactory.createRevisionDTO(revisionManager.getRevision(serviceNode.getIdentifier())));
         entity.setId(serviceNode.getIdentifier());
 
         final Authorizable authorizable = authorizableLookup.getControllerService(serviceNode.getIdentifier()).getAuthorizable();
-        final PermissionsDTO permissionsDto = dtoFactory.createPermissionsDto(authorizable, user);
+        final PermissionsDTO permissionsDto = dtoFactory.createPermissionsDto(authorizable);
         entity.setPermissions(permissionsDto);
 
         final AffectedComponentDTO dto = new AffectedComponentDTO();
@@ -4220,13 +4155,13 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
         return entity;
     }
 
-    private AffectedComponentEntity createAffectedComponentEntity(final InstantiatedVersionedComponent instance, final String componentTypeName, final String componentState, final NiFiUser user) {
+    private AffectedComponentEntity createAffectedComponentEntity(final InstantiatedVersionedComponent instance, final String componentTypeName, final String componentState) {
         final AffectedComponentEntity entity = new AffectedComponentEntity();
         entity.setRevision(dtoFactory.createRevisionDTO(revisionManager.getRevision(instance.getInstanceId())));
         entity.setId(instance.getInstanceId());
 
         final Authorizable authorizable = getAuthorizable(componentTypeName, instance);
-        final PermissionsDTO permissionsDto = dtoFactory.createPermissionsDto(authorizable, user);
+        final PermissionsDTO permissionsDto = dtoFactory.createPermissionsDto(authorizable);
         entity.setPermissions(permissionsDto);
 
         final AffectedComponentDTO dto = new AffectedComponentDTO();
@@ -4367,9 +4302,11 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
     }
 
     @Override
-    public ProcessGroupEntity updateProcessGroupContents(final NiFiUser user, final Revision revision, final String groupId, final VersionControlInformationDTO versionControlInfo,
+    public ProcessGroupEntity updateProcessGroupContents(final Revision revision, final String groupId, final VersionControlInformationDTO versionControlInfo,
         final VersionedFlowSnapshot proposedFlowSnapshot, final String componentIdSeed, final boolean verifyNotModified, final boolean updateSettings, final boolean updateDescendantVersionedFlows) {
 
+        final NiFiUser user = NiFiUserUtils.getNiFiUser();
+
         final ProcessGroup processGroup = processGroupDAO.getProcessGroup(groupId);
         final List<Revision> revisions = getComponentRevisions(processGroup, false);
         revisions.add(revision);
@@ -4380,7 +4317,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
             @Override
             public RevisionUpdate<ProcessGroupDTO> update() {
                 // update the Process Group
-                processGroupDAO.updateProcessGroupFlow(groupId, user, proposedFlowSnapshot, versionControlInfo, componentIdSeed, verifyNotModified, updateSettings, updateDescendantVersionedFlows);
+                processGroupDAO.updateProcessGroupFlow(groupId, proposedFlowSnapshot, versionControlInfo, componentIdSeed, verifyNotModified, updateSettings, updateDescendantVersionedFlows);
 
                 // update the revisions
                 final Set<Revision> updatedRevisions = revisions.stream()
@@ -4546,11 +4483,6 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
         return history;
     }
 
-    private ControllerServiceEntity createControllerServiceEntity(final String serviceId, final NiFiUser user) {
-        final ControllerServiceNode serviceNode = controllerServiceDAO.getControllerService(serviceId);
-        return createControllerServiceEntity(serviceNode, Collections.emptySet(), user);
-    }
-
     @Override
     public ProcessorDiagnosticsEntity getProcessorDiagnostics(final String id) {
         final ProcessorNode processor = processorDAO.getProcessor(id);
@@ -4558,7 +4490,10 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
 
         // Generate Processor Diagnostics
         final NiFiUser user = NiFiUserUtils.getNiFiUser();
-        final ProcessorDiagnosticsDTO dto = controllerFacade.getProcessorDiagnostics(processor, processorStatus, bulletinRepository, serviceId -> createControllerServiceEntity(serviceId, user));
+        final ProcessorDiagnosticsDTO dto = controllerFacade.getProcessorDiagnostics(processor, processorStatus, bulletinRepository, serviceId -> {
+            final ControllerServiceNode serviceNode = controllerServiceDAO.getControllerService(serviceId);
+            return createControllerServiceEntity(serviceNode, Collections.emptySet());
+        });
 
         // Filter anything out of diagnostics that the user is not authorized to see.
         final List<JVMDiagnosticsSnapshotDTO> jvmDiagnosticsSnaphots = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/nifi/blob/b7272e3f/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 2cbcf56..835aa7e 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
@@ -1092,15 +1092,14 @@ public class ProcessGroupResource extends ApplicationResource {
      * @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 VariableRegistryUpdateRequest updateRequest, final Pause pause, final NiFiUser user) {
+                                                        final VariableRegistryUpdateRequest updateRequest, final Pause pause) {
 
         boolean continuePolling = true;
         while (continuePolling) {
-            final Set<ControllerServiceEntity> serviceEntities = serviceFacade.getControllerServices(groupId, false, true, user);
+            final Set<ControllerServiceEntity> serviceEntities = serviceFacade.getControllerServices(groupId, false, true);
 
             // update the affected controller services
             updateAffectedControllerServices(serviceEntities, updateRequest);
@@ -1171,16 +1170,20 @@ public class ProcessGroupResource extends ApplicationResource {
             @Override
             public void run() {
                 try {
+                    // set the user authentication token
+                    final Authentication authentication = new NiFiAuthenticationToken(new NiFiUserDetails(user));
+                    SecurityContextHolder.getContext().setAuthentication(authentication);
+
                     // Stop processors
                     performUpdateVariableRegistryStep(groupId, updateRequest, updateRequest.getStopProcessorsStep(), "Stopping Processors",
-                        () -> stopProcessors(user, updateRequest, groupId, processorRevisionMap, pause));
+                        () -> stopProcessors(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));
+                        () -> disableControllerServices(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);
@@ -1188,17 +1191,17 @@ public class ProcessGroupResource extends ApplicationResource {
                     // Apply the updates
                     performUpdateVariableRegistryStep(groupId, updateRequest, updateRequest.getApplyUpdatesStep(), "Applying updates to Variable Registry",
                         () -> {
-                            final VariableRegistryEntity entity = serviceFacade.updateVariableRegistry(user, requestRevision, requestEntity.getVariableRegistry());
+                            final VariableRegistryEntity entity = serviceFacade.updateVariableRegistry(requestRevision, requestEntity.getVariableRegistry());
                             updateRequest.setProcessGroupRevision(entity.getProcessGroupRevision());
                         });
 
                     // Re-enable the controller services
                     performUpdateVariableRegistryStep(groupId, updateRequest, updateRequest.getEnableServicesStep(), "Re-enabling Controller Services",
-                        () -> enableControllerServices(user, updateRequest, groupId, updatedServiceRevisionMap, pause));
+                        () -> enableControllerServices(updateRequest, groupId, updatedServiceRevisionMap, pause));
 
                     // Restart processors
                     performUpdateVariableRegistryStep(groupId, updateRequest, updateRequest.getStartProcessorsStep(), "Restarting Processors",
-                        () -> startProcessors(user, updateRequest, groupId, updatedProcessorRevisionMap, pause));
+                        () -> startProcessors(updateRequest, groupId, updatedProcessorRevisionMap, pause));
 
                     // Set complete
                     updateRequest.setComplete(true);
@@ -1208,6 +1211,9 @@ public class ProcessGroupResource extends ApplicationResource {
 
                     updateRequest.setComplete(true);
                     updateRequest.setFailureReason("An unexpected error has occurred: " + e);
+                } finally {
+                    // clear the authentication token
+                    SecurityContextHolder.getContext().setAuthentication(null);
                 }
             }
         };
@@ -1256,7 +1262,7 @@ public class ProcessGroupResource extends ApplicationResource {
         request.setLastUpdated(new Date());
     }
 
-    private void stopProcessors(final NiFiUser user, final VariableRegistryUpdateRequest updateRequest, final String processGroupId,
+    private void stopProcessors(final VariableRegistryUpdateRequest updateRequest, final String processGroupId,
         final Map<String, Revision> processorRevisions, final Pause pause) {
 
         if (processorRevisions.isEmpty()) {
@@ -1264,21 +1270,21 @@ public class ProcessGroupResource extends ApplicationResource {
         }
 
         serviceFacade.verifyScheduleComponents(processGroupId, ScheduledState.STOPPED, processorRevisions.keySet());
-        serviceFacade.scheduleComponents(user, processGroupId, ScheduledState.STOPPED, processorRevisions);
+        serviceFacade.scheduleComponents(processGroupId, ScheduledState.STOPPED, processorRevisions);
         waitForLocalProcessor(processGroupId, processorRevisions.keySet(), ScheduledState.STOPPED, updateRequest, pause);
     }
 
-    private void startProcessors(final NiFiUser user, final VariableRegistryUpdateRequest request, final String processGroupId, final Map<String, Revision> processorRevisions, final Pause pause) {
+    private void startProcessors(final VariableRegistryUpdateRequest request, 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);
+        serviceFacade.scheduleComponents(processGroupId, ScheduledState.RUNNING, processorRevisions);
         waitForLocalProcessor(processGroupId, processorRevisions.keySet(), ScheduledState.RUNNING, request, pause);
     }
 
-    private void disableControllerServices(final NiFiUser user, final VariableRegistryUpdateRequest updateRequest, final String processGroupId,
+    private void disableControllerServices(final VariableRegistryUpdateRequest updateRequest, final String processGroupId,
                                            final Map<String, Revision> serviceRevisions, final Pause pause) {
 
         if (serviceRevisions.isEmpty()) {
@@ -1286,11 +1292,11 @@ public class ProcessGroupResource extends ApplicationResource {
         }
 
         serviceFacade.verifyActivateControllerServices(processGroupId, ControllerServiceState.DISABLED, serviceRevisions.keySet());
-        serviceFacade.activateControllerServices(user, processGroupId, ControllerServiceState.DISABLED, serviceRevisions);
-        waitForLocalControllerServiceStatus(processGroupId, serviceRevisions.keySet(), ControllerServiceState.DISABLED, updateRequest, pause, user);
+        serviceFacade.activateControllerServices(processGroupId, ControllerServiceState.DISABLED, serviceRevisions);
+        waitForLocalControllerServiceStatus(processGroupId, serviceRevisions.keySet(), ControllerServiceState.DISABLED, updateRequest, pause);
     }
 
-    private void enableControllerServices(final NiFiUser user, final VariableRegistryUpdateRequest updateRequest, final String processGroupId,
+    private void enableControllerServices(final VariableRegistryUpdateRequest updateRequest, final String processGroupId,
                                           final Map<String, Revision> serviceRevisions, final Pause pause) {
 
         if (serviceRevisions.isEmpty()) {
@@ -1298,8 +1304,8 @@ public class ProcessGroupResource extends ApplicationResource {
         }
 
         serviceFacade.verifyActivateControllerServices(processGroupId, ControllerServiceState.ENABLED, serviceRevisions.keySet());
-        serviceFacade.activateControllerServices(user, processGroupId, ControllerServiceState.ENABLED, serviceRevisions);
-        waitForLocalControllerServiceStatus(processGroupId, serviceRevisions.keySet(), ControllerServiceState.ENABLED, updateRequest, pause, user);
+        serviceFacade.activateControllerServices(processGroupId, ControllerServiceState.ENABLED, serviceRevisions);
+        waitForLocalControllerServiceStatus(processGroupId, serviceRevisions.keySet(), ControllerServiceState.ENABLED, updateRequest, pause);
     }
 
 
@@ -1728,8 +1734,8 @@ public class ProcessGroupResource extends ApplicationResource {
                         // placed the Process Group. However, we do want to use the name of the Process Group that is in the Flow Contents.
                         // To accomplish this, we call updateProcessGroupContents() passing 'true' for the updateSettings flag but null out the position.
                         flowSnapshot.getFlowContents().setPosition(null);
-                        entity = serviceFacade.updateProcessGroupContents(NiFiUserUtils.getNiFiUser(), newGroupRevision, newGroupId,
-                            versionControlInfo, flowSnapshot, getIdGenerationSeed().orElse(null), false, true, true);
+                        entity = serviceFacade.updateProcessGroupContents(newGroupRevision, newGroupId, versionControlInfo, flowSnapshot,
+                                getIdGenerationSeed().orElse(null), false, true, true);
                     }
 
                     populateRemainingProcessGroupEntityContent(entity);

http://git-wip-us.apache.org/repos/asf/nifi/blob/b7272e3f/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/VersionsResource.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/VersionsResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/VersionsResource.java
index f1a0053..b80b4d3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/VersionsResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/VersionsResource.java
@@ -794,8 +794,7 @@ public class VersionsResource extends ApplicationResource {
                 final VersionedFlowState flowState = snapshotMetadata.getVersion() == flow.getVersionCount() ? VersionedFlowState.UP_TO_DATE : VersionedFlowState.STALE;
                 versionControlInfoDto.setState(flowState.name());
 
-                final NiFiUser user = NiFiUserUtils.getNiFiUser();
-                final ProcessGroupEntity updatedGroup = serviceFacade.updateProcessGroupContents(user, rev, groupId, versionControlInfoDto, flowSnapshot, getIdGenerationSeed().orElse(null), false,
+                final ProcessGroupEntity updatedGroup = serviceFacade.updateProcessGroupContents(rev, groupId, versionControlInfoDto, flowSnapshot, getIdGenerationSeed().orElse(null), false,
                     true, entity.getUpdateDescendantVersionedFlows());
                 final VersionControlInformationDTO updatedVci = updatedGroup.getComponent().getVersionControlInformation();
 
@@ -1095,7 +1094,7 @@ public class VersionsResource extends ApplicationResource {
         BundleUtils.discoverCompatibleBundles(flowSnapshot.getFlowContents());
 
         // Step 1: Determine which components will be affected by updating the version
-        final Set<AffectedComponentEntity> affectedComponents = serviceFacade.getComponentsAffectedByVersionChange(groupId, flowSnapshot, user);
+        final Set<AffectedComponentEntity> affectedComponents = serviceFacade.getComponentsAffectedByVersionChange(groupId, flowSnapshot);
 
         // build a request wrapper
         final InitiateChangeFlowVersionRequestWrapper requestWrapper = new InitiateChangeFlowVersionRequestWrapper(requestEntity, componentLifecycle, getAbsolutePath(), affectedComponents,
@@ -1137,7 +1136,7 @@ public class VersionsResource extends ApplicationResource {
                 final Consumer<AsynchronousWebRequest<VersionControlInformationEntity>> updateTask = vcur -> {
                     try {
                         final VersionControlInformationEntity updatedVersionControlEntity = updateFlowVersion(groupId, wrapper.getComponentLifecycle(), wrapper.getExampleUri(),
-                            wrapper.getAffectedComponents(), user, wrapper.isReplicateRequest(), revision, wrapper.getVersionControlInformationEntity(), wrapper.getFlowSnapshot(), request,
+                            wrapper.getAffectedComponents(), wrapper.isReplicateRequest(), revision, wrapper.getVersionControlInformationEntity(), wrapper.getFlowSnapshot(), request,
                             idGenerationSeed, true, true);
 
                         vcur.markComplete(updatedVersionControlEntity);
@@ -1252,7 +1251,7 @@ public class VersionsResource extends ApplicationResource {
         BundleUtils.discoverCompatibleBundles(flowSnapshot.getFlowContents());
 
         // Step 1: Determine which components will be affected by updating the version
-        final Set<AffectedComponentEntity> affectedComponents = serviceFacade.getComponentsAffectedByVersionChange(groupId, flowSnapshot, user);
+        final Set<AffectedComponentEntity> affectedComponents = serviceFacade.getComponentsAffectedByVersionChange(groupId, flowSnapshot);
 
         // build a request wrapper
         final InitiateChangeFlowVersionRequestWrapper requestWrapper = new InitiateChangeFlowVersionRequestWrapper(requestEntity, componentLifecycle, getAbsolutePath(), affectedComponents,
@@ -1316,7 +1315,7 @@ public class VersionsResource extends ApplicationResource {
                 final Consumer<AsynchronousWebRequest<VersionControlInformationEntity>> updateTask = vcur -> {
                     try {
                         final VersionControlInformationEntity updatedVersionControlEntity = updateFlowVersion(groupId, wrapper.getComponentLifecycle(), wrapper.getExampleUri(),
-                            wrapper.getAffectedComponents(), user, wrapper.isReplicateRequest(), revision, versionControlInformationEntity, wrapper.getFlowSnapshot(), request,
+                            wrapper.getAffectedComponents(), wrapper.isReplicateRequest(), revision, versionControlInformationEntity, wrapper.getFlowSnapshot(), request,
                             idGenerationSeed, false, true);
 
                         vcur.markComplete(updatedVersionControlEntity);
@@ -1355,7 +1354,7 @@ public class VersionsResource extends ApplicationResource {
     }
 
     private VersionControlInformationEntity updateFlowVersion(final String groupId, final ComponentLifecycle componentLifecycle, final URI exampleUri,
-        final Set<AffectedComponentEntity> affectedComponents, final NiFiUser user, final boolean replicateRequest, final Revision revision, final VersionControlInformationEntity requestEntity,
+        final Set<AffectedComponentEntity> affectedComponents, final boolean replicateRequest, final Revision revision, final VersionControlInformationEntity requestEntity,
         final VersionedFlowSnapshot flowSnapshot, final AsynchronousWebRequest<VersionControlInformationEntity> asyncRequest, final String idGenerationSeed,
         final boolean verifyNotModified, final boolean updateDescendantVersionedFlows) throws LifecycleManagementException, ResumeFlowException {
 
@@ -1375,7 +1374,7 @@ public class VersionsResource extends ApplicationResource {
         logger.info("Stopping {} Processors", runningComponents.size());
         final CancellableTimedPause stopComponentsPause = new CancellableTimedPause(250, Long.MAX_VALUE, TimeUnit.MILLISECONDS);
         asyncRequest.setCancelCallback(stopComponentsPause::cancel);
-        componentLifecycle.scheduleComponents(exampleUri, user, groupId, runningComponents, ScheduledState.STOPPED, stopComponentsPause);
+        componentLifecycle.scheduleComponents(exampleUri, groupId, runningComponents, ScheduledState.STOPPED, stopComponentsPause);
 
         if (asyncRequest.isCancelled()) {
             return null;
@@ -1391,7 +1390,7 @@ public class VersionsResource extends ApplicationResource {
         logger.info("Disabling {} Controller Services", enabledServices.size());
         final CancellableTimedPause disableServicesPause = new CancellableTimedPause(250, Long.MAX_VALUE, TimeUnit.MILLISECONDS);
         asyncRequest.setCancelCallback(disableServicesPause::cancel);
-        componentLifecycle.activateControllerServices(exampleUri, user, groupId, enabledServices, ControllerServiceState.DISABLED, disableServicesPause);
+        componentLifecycle.activateControllerServices(exampleUri, groupId, enabledServices, ControllerServiceState.DISABLED, disableServicesPause);
 
         if (asyncRequest.isCancelled()) {
             return null;
@@ -1399,10 +1398,12 @@ public class VersionsResource extends ApplicationResource {
         asyncRequest.update(new Date(), "Updating Flow", 40);
 
         logger.info("Updating Process Group with ID {} to version {} of the Versioned Flow", groupId, flowSnapshot.getSnapshotMetadata().getVersion());
+
         // If replicating request, steps 10-12 are performed on each node individually, and this is accomplished
         // by replicating a PUT to /nifi-api/versions/process-groups/{groupId}
         try {
             if (replicateRequest) {
+                final NiFiUser user = NiFiUserUtils.getNiFiUser();
 
                 final URI updateUri;
                 try {
@@ -1470,7 +1471,7 @@ public class VersionsResource extends ApplicationResource {
                 vci.setVersion(metadata.getVersion());
                 vci.setState(flowSnapshot.isLatest() ? VersionedFlowState.UP_TO_DATE.name() : VersionedFlowState.STALE.name());
 
-                serviceFacade.updateProcessGroupContents(user, revision, groupId, vci, flowSnapshot, idGenerationSeed, verifyNotModified, false, updateDescendantVersionedFlows);
+                serviceFacade.updateProcessGroupContents(revision, groupId, vci, flowSnapshot, idGenerationSeed, verifyNotModified, false, updateDescendantVersionedFlows);
             }
         } finally {
             if (!asyncRequest.isCancelled()) {
@@ -1483,11 +1484,11 @@ public class VersionsResource extends ApplicationResource {
                 // Step 13. Re-enable all disabled controller services
                 final CancellableTimedPause enableServicesPause = new CancellableTimedPause(250, Long.MAX_VALUE, TimeUnit.MILLISECONDS);
                 asyncRequest.setCancelCallback(enableServicesPause::cancel);
-                final Set<AffectedComponentEntity> servicesToEnable = getUpdatedEntities(enabledServices, user);
+                final Set<AffectedComponentEntity> servicesToEnable = getUpdatedEntities(enabledServices);
                 logger.info("Successfully updated flow; re-enabling {} Controller Services", servicesToEnable.size());
 
                 try {
-                    componentLifecycle.activateControllerServices(exampleUri, user, groupId, servicesToEnable, ControllerServiceState.ENABLED, enableServicesPause);
+                    componentLifecycle.activateControllerServices(exampleUri, groupId, servicesToEnable, ControllerServiceState.ENABLED, enableServicesPause);
                 } catch (final IllegalStateException ise) {
                     // Component Lifecycle will re-enable the Controller Services only if they are valid. If IllegalStateException gets thrown, we need to provide
                     // a more intelligent error message as to exactly what happened, rather than indicate that the flow could not be updated.
@@ -1503,7 +1504,7 @@ public class VersionsResource extends ApplicationResource {
                 asyncRequest.update(new Date(), "Restarting Processors", 80);
 
                 // Step 14. Restart all components
-                final Set<AffectedComponentEntity> componentsToStart = getUpdatedEntities(runningComponents, user);
+                final Set<AffectedComponentEntity> componentsToStart = getUpdatedEntities(runningComponents);
 
                 // If there are any Remote Group Ports that are supposed to be started and have no connections, we want to remove those from our Set.
                 // This will happen if the Remote Group Port is transmitting when the version change happens but the new flow version does not have
@@ -1539,7 +1540,7 @@ public class VersionsResource extends ApplicationResource {
                 logger.info("Restarting {} Processors", componentsToStart.size());
 
                 try {
-                    componentLifecycle.scheduleComponents(exampleUri, user, groupId, componentsToStart, ScheduledState.RUNNING, startComponentsPause);
+                    componentLifecycle.scheduleComponents(exampleUri, groupId, componentsToStart, ScheduledState.RUNNING, startComponentsPause);
                 } catch (final IllegalStateException ise) {
                     // Component Lifecycle will restart the Processors only if they are valid. If IllegalStateException gets thrown, we need to provide
                     // a more intelligent error message as to exactly what happened, rather than indicate that the flow could not be updated.
@@ -1576,12 +1577,12 @@ public class VersionsResource extends ApplicationResource {
     }
 
 
-    private Set<AffectedComponentEntity> getUpdatedEntities(final Set<AffectedComponentEntity> originalEntities, final NiFiUser user) {
+    private Set<AffectedComponentEntity> getUpdatedEntities(final Set<AffectedComponentEntity> originalEntities) {
         final Set<AffectedComponentEntity> entities = new LinkedHashSet<>();
 
         for (final AffectedComponentEntity original : originalEntities) {
             try {
-                final AffectedComponentEntity updatedEntity = AffectedComponentUtils.updateEntity(original, serviceFacade, dtoFactory, user);
+                final AffectedComponentEntity updatedEntity = AffectedComponentUtils.updateEntity(original, serviceFacade, dtoFactory);
                 if (updatedEntity != null) {
                     entities.add(updatedEntity);
                 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/b7272e3f/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/concurrent/AsyncRequestManager.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/concurrent/AsyncRequestManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/concurrent/AsyncRequestManager.java
index 5dcb125..3f7ad63 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/concurrent/AsyncRequestManager.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/concurrent/AsyncRequestManager.java
@@ -17,6 +17,15 @@
 
 package org.apache.nifi.web.api.concurrent;
 
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.authorization.user.NiFiUserDetails;
+import org.apache.nifi.web.ResourceNotFoundException;
+import org.apache.nifi.web.security.token.NiFiAuthenticationToken;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.core.context.SecurityContextHolder;
+
 import java.util.Date;
 import java.util.List;
 import java.util.Map;
@@ -33,11 +42,6 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.function.Consumer;
 import java.util.stream.Collectors;
 
-import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.web.ResourceNotFoundException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 public class AsyncRequestManager<T> implements RequestManager<T> {
     private static final Logger logger = LoggerFactory.getLogger(AsyncRequestManager.class);
 
@@ -106,10 +110,17 @@ public class AsyncRequestManager<T> implements RequestManager<T> {
             @Override
             public void run() {
                 try {
+                    // set the user authentication token
+                    final Authentication authentication = new NiFiAuthenticationToken(new NiFiUserDetails(request.getUser()));
+                    SecurityContextHolder.getContext().setAuthentication(authentication);
+
                     task.accept(request);
                 } catch (final Exception e) {
                     logger.error("Failed to perform asynchronous task", e);
                     request.setFailureReason("Encountered unexpected error when performing asynchronous task: " + e);
+                } finally {
+                    // clear the authentication token
+                    SecurityContextHolder.getContext().setAuthentication(null);
                 }
             }
         });

http://git-wip-us.apache.org/repos/asf/nifi/blob/b7272e3f/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 3aac227..85d2876 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
@@ -2488,14 +2488,10 @@ public final class DtoFactory {
     }
 
     public Set<AffectedComponentEntity> createAffectedComponentEntities(final Set<ConfiguredComponent> affectedComponents, final RevisionManager revisionManager) {
-        return createAffectedComponentEntities(affectedComponents, revisionManager, NiFiUserUtils.getNiFiUser());
-    }
-
-    public Set<AffectedComponentEntity> createAffectedComponentEntities(final Set<ConfiguredComponent> affectedComponents, final RevisionManager revisionManager, final NiFiUser user) {
         return affectedComponents.stream()
                 .map(component -> {
                     final AffectedComponentDTO affectedComponent = createAffectedComponentDto(component);
-                    final PermissionsDTO permissions = createPermissionsDto(component, user);
+                    final PermissionsDTO permissions = createPermissionsDto(component);
                     final RevisionDTO revision = createRevisionDTO(revisionManager.getRevision(component.getIdentifier()));
                     return entityFactory.createAffectedComponentEntity(affectedComponent, revision, permissions);
                 })
@@ -2503,10 +2499,6 @@ public final class DtoFactory {
     }
 
     public VariableRegistryDTO createVariableRegistryDto(final ProcessGroup processGroup, final RevisionManager revisionManager) {
-        return createVariableRegistryDto(processGroup, revisionManager, NiFiUserUtils.getNiFiUser());
-    }
-
-    public VariableRegistryDTO createVariableRegistryDto(final ProcessGroup processGroup, final RevisionManager revisionManager, final NiFiUser user) {
         final ComponentVariableRegistry variableRegistry = processGroup.getVariableRegistry();
 
         final List<String> variableNames = variableRegistry.getVariableMap().keySet().stream()
@@ -2521,7 +2513,7 @@ public final class DtoFactory {
             variableDto.setValue(variableRegistry.getVariableValue(variableName));
             variableDto.setProcessGroupId(processGroup.getIdentifier());
 
-            final Set<AffectedComponentEntity> affectedComponentEntities = createAffectedComponentEntities(processGroup.getComponentsAffectedByVariable(variableName), revisionManager, user);
+            final Set<AffectedComponentEntity> affectedComponentEntities = createAffectedComponentEntities(processGroup.getComponentsAffectedByVariable(variableName), revisionManager);
 
             boolean canWrite = true;
             for (final AffectedComponentEntity affectedComponent : affectedComponentEntities) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/b7272e3f/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 7582420..36fd7dc 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,7 +16,6 @@
  */
 package org.apache.nifi.web.dao;
 
-import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.service.ControllerServiceState;
 import org.apache.nifi.groups.ProcessGroup;
@@ -93,10 +92,11 @@ public interface ProcessGroupDAO {
     /**
      * Enables or disables the controller services in the specified process group
      *
+     * @param groupId id
      * @param state the desired state
      * @param serviceIds the ID's of the services to enable or disable
      */
-    Future<Void> activateControllerServices(ControllerServiceState state, Collection<String> serviceIds);
+    Future<Void> activateControllerServices(String groupId, ControllerServiceState state, Collection<String> serviceIds);
 
     /**
      * Updates the specified process group.
@@ -118,7 +118,7 @@ public interface ProcessGroupDAO {
      *            update the contents of that Process Group
      * @return the process group
      */
-    ProcessGroup updateProcessGroupFlow(String groupId, NiFiUser user, VersionedFlowSnapshot proposedSnapshot, VersionControlInformationDTO versionControlInformation, String componentIdSeed,
+    ProcessGroup updateProcessGroupFlow(String groupId, VersionedFlowSnapshot proposedSnapshot, VersionControlInformationDTO versionControlInformation, String componentIdSeed,
                                         boolean verifyNotModified, boolean updateSettings, boolean updateDescendantVersionedFlows);
 
     /**
@@ -141,11 +141,10 @@ public interface ProcessGroupDAO {
     /**
      * Updates the specified variable registry
      *
-     * @param user the user performing the update
      * @param variableRegistry the Variable Registry
      * @return the Process Group that was updated
      */
-    ProcessGroup updateVariableRegistry(NiFiUser user, VariableRegistryDTO variableRegistry);
+    ProcessGroup updateVariableRegistry(VariableRegistryDTO variableRegistry);
 
     /**
      * Verifies that the specified updates to a current Process Group can be applied at this time

http://git-wip-us.apache.org/repos/asf/nifi/blob/b7272e3f/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 4744324..ca47764 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,7 +16,6 @@
  */
 package org.apache.nifi.web.dao.impl;
 
-import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.connectable.Position;
@@ -197,7 +196,7 @@ public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGrou
     }
 
     @Override
-    public Future<Void> activateControllerServices(final ControllerServiceState state, final Collection<String> serviceIds) {
+    public Future<Void> activateControllerServices(final String groupId, final ControllerServiceState state, final Collection<String> serviceIds) {
         final List<ControllerServiceNode> serviceNodes = serviceIds.stream()
             .map(flowController::getControllerServiceNode)
             .collect(Collectors.toList());
@@ -266,7 +265,7 @@ public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGrou
     }
 
     @Override
-    public ProcessGroup updateProcessGroupFlow(final String groupId, final NiFiUser user, final VersionedFlowSnapshot proposedSnapshot, final VersionControlInformationDTO versionControlInformation,
+    public ProcessGroup updateProcessGroupFlow(final String groupId, final VersionedFlowSnapshot proposedSnapshot, final VersionControlInformationDTO versionControlInformation,
                                                final String componentIdSeed, final boolean verifyNotModified, final boolean updateSettings, final boolean updateDescendantVersionedFlows) {
 
         final ProcessGroup group = locateProcessGroup(flowController, groupId);
@@ -284,7 +283,7 @@ public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGrou
     }
 
     @Override
-    public ProcessGroup updateVariableRegistry(final NiFiUser user, final VariableRegistryDTO variableRegistry) {
+    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());

http://git-wip-us.apache.org/repos/asf/nifi/blob/b7272e3f/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/AffectedComponentUtils.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/AffectedComponentUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/AffectedComponentUtils.java
index 05aa4f1..378c8ed 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/AffectedComponentUtils.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/AffectedComponentUtils.java
@@ -17,7 +17,6 @@
 
 package org.apache.nifi.web.util;
 
-import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.web.NiFiServiceFacade;
 import org.apache.nifi.web.api.dto.AffectedComponentDTO;
 import org.apache.nifi.web.api.dto.DtoFactory;
@@ -33,26 +32,25 @@ import java.util.Optional;
 
 public class AffectedComponentUtils {
 
-    public static AffectedComponentEntity updateEntity(final AffectedComponentEntity componentEntity, final NiFiServiceFacade serviceFacade,
-                final DtoFactory dtoFactory, final NiFiUser user) {
+    public static AffectedComponentEntity updateEntity(final AffectedComponentEntity componentEntity, final NiFiServiceFacade serviceFacade, final DtoFactory dtoFactory) {
 
         switch (componentEntity.getComponent().getReferenceType()) {
             case AffectedComponentDTO.COMPONENT_TYPE_PROCESSOR:
-                final ProcessorEntity procEntity = serviceFacade.getProcessor(componentEntity.getId(), user);
+                final ProcessorEntity procEntity = serviceFacade.getProcessor(componentEntity.getId());
                 return dtoFactory.createAffectedComponentEntity(procEntity);
             case AffectedComponentDTO.COMPONENT_TYPE_INPUT_PORT: {
-                final PortEntity portEntity = serviceFacade.getInputPort(componentEntity.getId(), user);
+                final PortEntity portEntity = serviceFacade.getInputPort(componentEntity.getId());
                 return dtoFactory.createAffectedComponentEntity(portEntity, AffectedComponentDTO.COMPONENT_TYPE_INPUT_PORT);
             }
             case AffectedComponentDTO.COMPONENT_TYPE_OUTPUT_PORT: {
-                final PortEntity portEntity = serviceFacade.getOutputPort(componentEntity.getId(), user);
+                final PortEntity portEntity = serviceFacade.getOutputPort(componentEntity.getId());
                 return dtoFactory.createAffectedComponentEntity(portEntity, AffectedComponentDTO.COMPONENT_TYPE_OUTPUT_PORT);
             }
             case AffectedComponentDTO.COMPONENT_TYPE_CONTROLLER_SERVICE:
-                final ControllerServiceEntity serviceEntity = serviceFacade.getControllerService(componentEntity.getId(), user);
+                final ControllerServiceEntity serviceEntity = serviceFacade.getControllerService(componentEntity.getId());
                 return dtoFactory.createAffectedComponentEntity(serviceEntity);
             case AffectedComponentDTO.COMPONENT_TYPE_REMOTE_INPUT_PORT: {
-                final RemoteProcessGroupEntity remoteGroupEntity = serviceFacade.getRemoteProcessGroup(componentEntity.getComponent().getProcessGroupId(), user);
+                final RemoteProcessGroupEntity remoteGroupEntity = serviceFacade.getRemoteProcessGroup(componentEntity.getComponent().getProcessGroupId());
                 final RemoteProcessGroupContentsDTO remoteGroupContents = remoteGroupEntity.getComponent().getContents();
                 final Optional<RemoteProcessGroupPortDTO> portDtoOption = remoteGroupContents.getInputPorts().stream()
                     .filter(port -> port.getId().equals(componentEntity.getId()))
@@ -65,7 +63,7 @@ public class AffectedComponentUtils {
                 break;
             }
             case AffectedComponentDTO.COMPONENT_TYPE_REMOTE_OUTPUT_PORT: {
-                final RemoteProcessGroupEntity remoteGroupEntity = serviceFacade.getRemoteProcessGroup(componentEntity.getComponent().getProcessGroupId(), user);
+                final RemoteProcessGroupEntity remoteGroupEntity = serviceFacade.getRemoteProcessGroup(componentEntity.getComponent().getProcessGroupId());
                 final RemoteProcessGroupContentsDTO remoteGroupContents = remoteGroupEntity.getComponent().getContents();
                 final Optional<RemoteProcessGroupPortDTO> portDtoOption = remoteGroupContents.getOutputPorts().stream()
                     .filter(port -> port.getId().equals(componentEntity.getId()))