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 2016/04/15 22:04:08 UTC

[16/22] nifi git commit: NIFI-1551: - Removing the AuthorityProvider. - Refactoring REST API in preparation for introduction of the Authorizer. - Updating UI accordingly. - Removing unneeded properties from nifi.properties. - Addressing comments from PR.

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java
index 359d45b..bfb064a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java
@@ -18,7 +18,7 @@ package org.apache.nifi.authorization;
 
 import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
 import org.apache.nifi.authorization.AuthorizationResult.Result;
-import org.apache.nifi.authorization.exception.ProviderCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
 import org.apache.nifi.authorization.resource.ResourceFactory;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.util.file.FileUtils;
@@ -111,20 +111,20 @@ public class FileAuthorizerTest {
         assertEquals(primary.length(), restore.length());
     }
 
-    @Test(expected = ProviderCreationException.class)
+    @Test(expected = AuthorizerCreationException.class)
     public void testPostConstructionWhenPrimaryDoesNotExist() throws Exception {
         writeAuthorizationsFile(restore, EMPTY_AUTHORIZATIONS_CONCISE);
         authorizer.onConfigured(configurationContext);
     }
 
-    @Test(expected = ProviderCreationException.class)
+    @Test(expected = AuthorizerCreationException.class)
     public void testPostConstructionWhenPrimaryDifferentThanRestore() throws Exception {
         writeAuthorizationsFile(primary, EMPTY_AUTHORIZATIONS);
         writeAuthorizationsFile(restore, EMPTY_AUTHORIZATIONS_CONCISE);
         authorizer.onConfigured(configurationContext);
     }
 
-    @Test(expected = ProviderCreationException.class)
+    @Test(expected = AuthorizerCreationException.class)
     public void testBadSchema() throws Exception {
         writeAuthorizationsFile(primary, BAD_SCHEMA_AUTHORIZATIONS);
         authorizer.onConfigured(configurationContext);
@@ -135,7 +135,8 @@ public class FileAuthorizerTest {
         writeAuthorizationsFile(primary, AUTHORIZATIONS);
         authorizer.onConfigured(configurationContext);
 
-        final AuthorizationRequest request = new AuthorizationRequest.Builder().resource(ResourceFactory.getFlowResource()).identity("user-1").action(RequestAction.READ).build();
+        final AuthorizationRequest request = new AuthorizationRequest.Builder().resource(ResourceFactory.getFlowResource()).identity("user-1").anonymous(false).accessAttempt(true).action(RequestAction
+            .READ).build();
         final AuthorizationResult result = authorizer.authorize(request);
         assertTrue(Result.Approved.equals(result.getResult()));
     }
@@ -145,7 +146,8 @@ public class FileAuthorizerTest {
         writeAuthorizationsFile(primary, AUTHORIZATIONS);
         authorizer.onConfigured(configurationContext);
 
-        final AuthorizationRequest request = new AuthorizationRequest.Builder().resource(ResourceFactory.getFlowResource()).identity("user-2").action(RequestAction.READ).build();
+        final AuthorizationRequest request =
+            new AuthorizationRequest.Builder().resource(ResourceFactory.getFlowResource()).identity("user-2").anonymous(false).accessAttempt(true).action(RequestAction.READ).build();
         final AuthorizationResult result = authorizer.authorize(request);
         assertFalse(Result.Approved.equals(result.getResult()));
     }
@@ -155,7 +157,8 @@ public class FileAuthorizerTest {
         writeAuthorizationsFile(primary, AUTHORIZATIONS);
         authorizer.onConfigured(configurationContext);
 
-        final AuthorizationRequest request = new AuthorizationRequest.Builder().resource(ResourceFactory.getFlowResource()).identity("user-1").action(RequestAction.WRITE).build();
+        final AuthorizationRequest request =
+            new AuthorizationRequest.Builder().resource(ResourceFactory.getFlowResource()).identity("user-1").anonymous(false).accessAttempt(true).action(RequestAction.WRITE).build();
         final AuthorizationResult result = authorizer.authorize(request);
         assertFalse(Result.Approved.equals(result.getResult()));
     }
@@ -167,7 +170,8 @@ public class FileAuthorizerTest {
         authorizer.onConfigured(configurationContext);
 
         // ensure the user currently does not have write access
-        final AuthorizationRequest request = new AuthorizationRequest.Builder().resource(ResourceFactory.getFlowResource()).identity("user-1").action(RequestAction.WRITE).build();
+        final AuthorizationRequest request =
+            new AuthorizationRequest.Builder().resource(ResourceFactory.getFlowResource()).identity("user-1").anonymous(false).accessAttempt(true).action(RequestAction.WRITE).build();
         AuthorizationResult result = authorizer.authorize(request);
         assertFalse(Result.Approved.equals(result.getResult()));
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
index 81ee5a0..ee59942 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
@@ -411,12 +411,28 @@ public interface ProcessGroup {
     Set<Connection> getConnections();
 
     /**
+     * @param id of the Connection
+     * @return the Connection with the given ID, if it exists as a child or
+     * descendant of this ProcessGroup. This performs a recursive search of all
+     * descendant ProcessGroups
+     */
+    Connection findConnection(String id);
+
+    /**
      * @return a List of all Connections contains within this ProcessGroup and
      * any child ProcessGroups
      */
     List<Connection> findAllConnections();
 
     /**
+     * @param id of the Funnel
+     * @return the Funnel with the given ID, if it exists as a child or
+     * descendant of this ProcessGroup. This performs a recursive search of all
+     * descendant ProcessGroups
+     */
+    Funnel findFunnel(String id);
+
+    /**
      * Adds the given RemoteProcessGroup to this ProcessGroup
      *
      * @param remoteGroup group to add
@@ -521,6 +537,14 @@ public interface ProcessGroup {
     List<ProcessorNode> findAllProcessors();
 
     /**
+     * @param id of the Label
+     * @return the Label with the given ID, if it exists as a child or
+     * descendant of this ProcessGroup. This performs a recursive search of all
+     * descendant ProcessGroups
+     */
+    Label findLabel(String id);
+
+    /**
      * @return a List of all Labels that are children or descendants of this
      * ProcessGroup. This performsn a recursive search of all descendant
      * ProcessGroups

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index 9f14354..75395b7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -53,7 +53,7 @@ import javax.net.ssl.SSLContext;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.action.Action;
 import org.apache.nifi.admin.service.AuditService;
-import org.apache.nifi.admin.service.UserService;
+import org.apache.nifi.admin.service.KeyService;
 import org.apache.nifi.annotation.lifecycle.OnAdded;
 import org.apache.nifi.annotation.lifecycle.OnConfigurationRestored;
 import org.apache.nifi.annotation.lifecycle.OnRemoved;
@@ -257,7 +257,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
     private final AtomicReference<CounterRepository> counterRepositoryRef;
     private final AtomicBoolean initialized = new AtomicBoolean(false);
     private final ControllerServiceProvider controllerServiceProvider;
-    private final UserService userService;
+    private final KeyService keyService;
     private final AuditService auditService;
     private final EventDrivenWorkerQueue eventDrivenWorkerQueue;
     private final ComponentStatusRepository componentStatusRepository;
@@ -354,13 +354,13 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
     public static FlowController createStandaloneInstance(
         final FlowFileEventRepository flowFileEventRepo,
         final NiFiProperties properties,
-        final UserService userService,
+        final KeyService keyService,
         final AuditService auditService,
         final StringEncryptor encryptor) {
         return new FlowController(
             flowFileEventRepo,
             properties,
-            userService,
+            keyService,
             auditService,
             encryptor,
             /* configuredForClustering */ false,
@@ -370,14 +370,14 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
     public static FlowController createClusteredInstance(
         final FlowFileEventRepository flowFileEventRepo,
         final NiFiProperties properties,
-        final UserService userService,
+        final KeyService keyService,
         final AuditService auditService,
         final StringEncryptor encryptor,
         final NodeProtocolSender protocolSender) {
         final FlowController flowController = new FlowController(
             flowFileEventRepo,
             properties,
-            userService,
+            keyService,
             auditService,
             encryptor,
             /* configuredForClustering */ true,
@@ -391,7 +391,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
     private FlowController(
         final FlowFileEventRepository flowFileEventRepo,
         final NiFiProperties properties,
-        final UserService userService,
+        final KeyService keyService,
         final AuditService auditService,
         final StringEncryptor encryptor,
         final boolean configuredForClustering,
@@ -447,7 +447,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
 
         startConnectablesAfterInitialization = new ArrayList<>();
         startRemoteGroupPortsAfterInitialization = new ArrayList<>();
-        this.userService = userService;
+        this.keyService = keyService;
         this.auditService = auditService;
 
         final String gracefulShutdownSecondsVal = properties.getProperty(GRACEFUL_SHUTDOWN_PERIOD);
@@ -1032,7 +1032,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         name = requireNonNull(name).intern();
         verifyPortIdDoesNotExist(id);
         return new StandardRootGroupPort(id, name, null, TransferDirection.RECEIVE, ConnectableType.INPUT_PORT,
-            userService, getBulletinRepository(), processScheduler, Boolean.TRUE.equals(isSiteToSiteSecure));
+            keyService, getBulletinRepository(), processScheduler, Boolean.TRUE.equals(isSiteToSiteSecure));
     }
 
     /**
@@ -1049,7 +1049,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         name = requireNonNull(name).intern();
         verifyPortIdDoesNotExist(id);
         return new StandardRootGroupPort(id, name, null, TransferDirection.SEND, ConnectableType.OUTPUT_PORT,
-            userService, getBulletinRepository(), processScheduler, Boolean.TRUE.equals(isSiteToSiteSecure));
+            keyService, getBulletinRepository(), processScheduler, Boolean.TRUE.equals(isSiteToSiteSecure));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
index 4646d55..71e51b6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
@@ -945,6 +945,27 @@ public final class StandardProcessGroup implements ProcessGroup {
     }
 
     @Override
+    public Connection findConnection(final String id) {
+        return findConnection(id, this);
+    }
+
+    private Connection findConnection(final String id, final ProcessGroup start) {
+        Connection connection = start.getConnection(id);
+        if (connection != null) {
+            return connection;
+        }
+
+        for (final ProcessGroup group : start.getProcessGroups()) {
+            connection = findConnection(id, group);
+            if (connection != null) {
+                return connection;
+            }
+        }
+
+        return null;
+    }
+
+    @Override
     public List<Connection> findAllConnections() {
         return findAllConnections(this);
     }
@@ -1476,6 +1497,27 @@ public final class StandardProcessGroup implements ProcessGroup {
     }
 
     @Override
+    public Label findLabel(final String id) {
+        return findLabel(id, this);
+    }
+
+    private Label findLabel(final String id, final ProcessGroup start) {
+        Label label = start.getLabel(id);
+        if (label != null) {
+            return label;
+        }
+
+        for (final ProcessGroup group : start.getProcessGroups()) {
+            label = findLabel(id, group);
+            if (label != null) {
+                return label;
+            }
+        }
+
+        return null;
+    }
+
+    @Override
     public List<Label> findAllLabels() {
         return findAllLabels(this);
     }
@@ -1603,6 +1645,27 @@ public final class StandardProcessGroup implements ProcessGroup {
     }
 
     @Override
+    public Funnel findFunnel(final String id) {
+        return findFunnel(id, this);
+    }
+
+    private Funnel findFunnel(final String id, final ProcessGroup start) {
+        Funnel funnel = start.getFunnel(id);
+        if (funnel != null) {
+            return funnel;
+        }
+
+        for (final ProcessGroup group : start.getProcessGroups()) {
+            funnel = findFunnel(id, group);
+            if (funnel != null) {
+                return funnel;
+            }
+        }
+
+        return null;
+    }
+
+    @Override
     public void removeFunnel(final Funnel funnel) {
         writeLock.lock();
         try {

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/FlowControllerFactoryBean.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/FlowControllerFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/FlowControllerFactoryBean.java
index c6c18c3..1c747d4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/FlowControllerFactoryBean.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/FlowControllerFactoryBean.java
@@ -17,7 +17,7 @@
 package org.apache.nifi.spring;
 
 import org.apache.nifi.admin.service.AuditService;
-import org.apache.nifi.admin.service.UserService;
+import org.apache.nifi.admin.service.KeyService;
 import org.apache.nifi.cluster.protocol.NodeProtocolSender;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.repository.FlowFileEventRepository;
@@ -38,7 +38,7 @@ public class FlowControllerFactoryBean implements FactoryBean, ApplicationContex
     private ApplicationContext applicationContext;
     private FlowController flowController;
     private NiFiProperties properties;
-    private UserService userService;
+    private KeyService keyService;
     private AuditService auditService;
     private StringEncryptor encryptor;
 
@@ -58,7 +58,7 @@ public class FlowControllerFactoryBean implements FactoryBean, ApplicationContex
                 flowController = FlowController.createClusteredInstance(
                         flowFileEventRepository,
                         properties,
-                        userService,
+                    keyService,
                         auditService,
                         encryptor,
                         nodeProtocolSender);
@@ -66,7 +66,7 @@ public class FlowControllerFactoryBean implements FactoryBean, ApplicationContex
                 flowController = FlowController.createStandaloneInstance(
                         flowFileEventRepository,
                         properties,
-                        userService,
+                    keyService,
                         auditService,
                         encryptor);
             }
@@ -95,8 +95,8 @@ public class FlowControllerFactoryBean implements FactoryBean, ApplicationContex
         this.properties = properties;
     }
 
-    public void setUserService(final UserService userService) {
-        this.userService = userService;
+    public void setKeyService(final KeyService keyService) {
+        this.keyService = keyService;
     }
 
     public void setEncryptor(final StringEncryptor encryptor) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml
index e841b24..c864ccf 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml
@@ -36,7 +36,7 @@
     <!-- flow controller -->
     <bean id="flowController" class="org.apache.nifi.spring.FlowControllerFactoryBean">
         <property name="properties" ref="nifiProperties"/>
-        <property name="userService" ref="userService" />
+        <property name="keyService" ref="keyService" />
         <property name="auditService" ref="auditService" />
         <property name="encryptor" ref="stringEncryptor" />
     </bean>

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
index b01b26c..9fb62df 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
@@ -25,7 +25,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.nifi.admin.service.AuditService;
-import org.apache.nifi.admin.service.UserService;
+import org.apache.nifi.admin.service.KeyService;
 import org.apache.nifi.cluster.protocol.StandardDataFlow;
 import org.apache.nifi.controller.repository.FlowFileEventRepository;
 import org.apache.nifi.util.NiFiProperties;
@@ -55,7 +55,7 @@ public class StandardFlowServiceTest {
     private FlowController flowController;
     private NiFiProperties properties;
     private FlowFileEventRepository mockFlowFileEventRepository;
-    private UserService mockUserService;
+    private KeyService mockKeyService;
     private AuditService mockAuditService;
     private StringEncryptor mockEncryptor;
 
@@ -68,9 +68,9 @@ public class StandardFlowServiceTest {
     public void setup() throws Exception {
         properties = NiFiProperties.getInstance();
         mockFlowFileEventRepository = mock(FlowFileEventRepository.class);
-        mockUserService = mock(UserService.class);
+        mockKeyService = mock(KeyService.class);
         mockAuditService = mock(AuditService.class);
-        flowController = FlowController.createStandaloneInstance(mockFlowFileEventRepository, properties, mockUserService, mockAuditService, mockEncryptor);
+        flowController = FlowController.createStandaloneInstance(mockFlowFileEventRepository, properties, mockKeyService, mockAuditService, mockEncryptor);
         flowService = StandardFlowService.createStandaloneInstance(flowController, properties, mockEncryptor);
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java
index 560c4cb..f98ed45 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java
@@ -37,7 +37,7 @@ import java.util.concurrent.locks.LockSupport;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.nifi.admin.service.AuditService;
-import org.apache.nifi.admin.service.UserService;
+import org.apache.nifi.admin.service.KeyService;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
@@ -640,7 +640,7 @@ public class TestProcessorLifecycle {
         properties.setProperty("nifi.remote.input.secure", "");
 
         return FlowController.createStandaloneInstance(mock(FlowFileEventRepository.class), properties,
-                mock(UserService.class), mock(AuditService.class), null);
+                mock(KeyService.class), mock(AuditService.class), null);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/conf/nifi.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/conf/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/conf/nifi.properties
index 78a649b..445f459 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/conf/nifi.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/conf/nifi.properties
@@ -82,11 +82,7 @@ nifi.security.truststore=
 nifi.security.truststoreType=
 nifi.security.truststorePasswd=
 nifi.security.needClientAuth=
-nifi.security.authorizedUsers.file=./target/conf/authorized-users.xml
-nifi.security.user.credential.cache.duration=24 hours
-nifi.security.user.authority.provider=nifi.authorization.FileAuthorizationProvider
-nifi.security.support.new.account.requests=
-nifi.security.default.user.roles=
+nifi.security.user.authorizer=
 
 # cluster common properties (cluster manager and nodes must have same values) #
 nifi.cluster.protocol.heartbeat.interval=5 sec

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/nifi-with-remote.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/nifi-with-remote.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/nifi-with-remote.properties
index e5b9a34..445f459 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/nifi-with-remote.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/nifi-with-remote.properties
@@ -82,11 +82,7 @@ nifi.security.truststore=
 nifi.security.truststoreType=
 nifi.security.truststorePasswd=
 nifi.security.needClientAuth=
-nifi.security.authorizedUsers.file=./target/conf/authorized-users.xml
-nifi.security.user.credential.cache.duration=24 hours
-nifi.security.user.authority.provider=org.apache.nifi.authorization.FileAuthorizationProvider
-nifi.security.support.new.account.requests=
-nifi.security.default.user.roles=
+nifi.security.user.authorizer=
 
 # cluster common properties (cluster manager and nodes must have same values) #
 nifi.cluster.protocol.heartbeat.interval=5 sec

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/nifi.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/nifi.properties
index d752c6d..210e7c6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/nifi.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/nifi.properties
@@ -82,11 +82,7 @@ nifi.security.truststore=
 nifi.security.truststoreType=
 nifi.security.truststorePasswd=
 nifi.security.needClientAuth=
-nifi.security.authorizedUsers.file=./target/conf/authorized-users.xml
-nifi.security.user.credential.cache.duration=24 hours
-nifi.security.user.authority.provider=org.apache.nifi.authorization.FileAuthorizationProvider
-nifi.security.support.new.account.requests=
-nifi.security.default.user.roles=
+nifi.security.user.authorizer=
 
 # cluster common properties (cluster manager and nodes must have same values) #
 nifi.cluster.protocol.heartbeat.interval=5 sec

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManager.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManager.java
index db0b35e..f06012c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManager.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManager.java
@@ -16,15 +16,8 @@
  */
 package org.apache.nifi.nar;
 
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.ServiceLoader;
-import java.util.Set;
 import org.apache.nifi.authentication.LoginIdentityProvider;
-
-import org.apache.nifi.authorization.AuthorityProvider;
+import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.repository.ContentRepository;
 import org.apache.nifi.controller.repository.FlowFileRepository;
@@ -34,10 +27,16 @@ import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.processor.Processor;
 import org.apache.nifi.provenance.ProvenanceEventRepository;
 import org.apache.nifi.reporting.ReportingTask;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.ServiceLoader;
+import java.util.Set;
+
 /**
  * Scans through the classpath to load all FlowFileProcessors, FlowFileComparators, and ReportingTasks using the service provider API and running through all classloaders (root, NARs).
  *
@@ -58,7 +57,7 @@ public class ExtensionManager {
         definitionMap.put(FlowFilePrioritizer.class, new HashSet<Class>());
         definitionMap.put(ReportingTask.class, new HashSet<Class>());
         definitionMap.put(ControllerService.class, new HashSet<Class>());
-        definitionMap.put(AuthorityProvider.class, new HashSet<Class>());
+        definitionMap.put(Authorizer.class, new HashSet<Class>());
         definitionMap.put(LoginIdentityProvider.class, new HashSet<Class>());
         definitionMap.put(ProvenanceEventRepository.class, new HashSet<Class>());
         definitionMap.put(ComponentStatusRepository.class, new HashSet<Class>());

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
index 9e9bd03..93f73eb 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
@@ -16,16 +16,8 @@
  */
 package org.apache.nifi.nar;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.util.ArrayList;
-import java.util.Enumeration;
-import java.util.List;
 import org.apache.nifi.authentication.LoginIdentityProvider;
-
-import org.apache.nifi.authorization.AuthorityProvider;
+import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.components.Validator;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.repository.ContentRepository;
@@ -40,6 +32,14 @@ import org.apache.nifi.processor.io.StreamCallback;
 import org.apache.nifi.provenance.ProvenanceEventRepository;
 import org.apache.nifi.reporting.ReportingTask;
 
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.List;
+
 /**
  * THREAD SAFE
  */
@@ -58,7 +58,7 @@ public class NarThreadContextClassLoader extends URLClassLoader {
         narSpecificClasses.add(OutputStreamCallback.class);
         narSpecificClasses.add(StreamCallback.class);
         narSpecificClasses.add(ControllerService.class);
-        narSpecificClasses.add(AuthorityProvider.class);
+        narSpecificClasses.add(Authorizer.class);
         narSpecificClasses.add(LoginIdentityProvider.class);
         narSpecificClasses.add(ProvenanceEventRepository.class);
         narSpecificClasses.add(ComponentStatusRepository.class);

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/NarUnpacker/conf/nifi.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/NarUnpacker/conf/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/NarUnpacker/conf/nifi.properties
index 103b7c1..69b5b09 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/NarUnpacker/conf/nifi.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/NarUnpacker/conf/nifi.properties
@@ -84,11 +84,7 @@ nifi.security.truststore=
 nifi.security.truststoreType=
 nifi.security.truststorePasswd=
 nifi.security.needClientAuth=
-nifi.security.authorizedUsers.file=./target/conf/authorized-users.xml
-nifi.security.user.credential.cache.duration=24 hours
-nifi.security.user.authority.provider=nifi.authorization.FileAuthorizationProvider
-nifi.security.support.new.account.requests=
-nifi.security.default.user.roles=
+nifi.security.user.authorizer=
 
 # cluster common properties (cluster manager and nodes must have same values) #
 nifi.cluster.protocol.heartbeat.interval=5 sec

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/authority-providers.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/authority-providers.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/authority-providers.xml
deleted file mode 100644
index cb68e15..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/authority-providers.xml
+++ /dev/null
@@ -1,43 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-<!--
-  Licensed to the Apache Software Foundation (ASF) under one or more
-  contributor license agreements.  See the NOTICE file distributed with
-  this work for additional information regarding copyright ownership.
-  The ASF licenses this file to You under the Apache License, Version 2.0
-  (the "License"); you may not use this file except in compliance with
-  the License.  You may obtain a copy of the License at
-      http://www.apache.org/licenses/LICENSE-2.0
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
--->
-<!--
-    This file lists the authority providers to use when running securely. In order
-    to use a specific provider it must be configured here and it's identifier
-    must be specified in the nifi.properties file.
--->
-<authorityProviders>
-    <provider>
-        <identifier>file-provider</identifier>
-        <class>org.apache.nifi.authorization.FileAuthorizationProvider</class>
-        <property name="Authorized Users File">./conf/authorized-users.xml</property>
-        <property name="Default User Roles"></property>
-    </provider>
-    
-    <!--<provider>
-        <identifier>cluster-ncm-provider</identifier>
-        <class>org.apache.nifi.cluster.authorization.ClusterManagerAuthorizationProvider</class>
-        <property name="Authority Provider Port"></property>
-        <property name="Authority Provider Threads">10</property>
-        <property name="Authorized Users File">./conf/authorized-users.xml</property>
-        <property name="Default User Roles"></property>
-    </provider>-->
-    
-    <!--<provider>
-        <identifier>cluster-node-provider</identifier>
-        <class>org.apache.nifi.cluster.authorization.NodeAuthorizationProvider</class>
-        <property name="Cluster Manager Authority Provider Port"></property>
-    </provider>-->
-</authorityProviders>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/authorized-users.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/authorized-users.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/authorized-users.xml
deleted file mode 100644
index 6b07165..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/authorized-users.xml
+++ /dev/null
@@ -1,57 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-<!--
-  Licensed to the Apache Software Foundation (ASF) under one or more
-  contributor license agreements.  See the NOTICE file distributed with
-  this work for additional information regarding copyright ownership.
-  The ASF licenses this file to You under the Apache License, Version 2.0
-  (the "License"); you may not use this file except in compliance with
-  the License.  You may obtain a copy of the License at
-      http://www.apache.org/licenses/LICENSE-2.0
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
--->
-<!--
-    This file lists all authorized users for this NiFi instance when using 
-    the FileAuthorizationProvider or ClusterManagerAuthorizationProvider. If one of
-    these providers is not in use then this file is not used. Refer to the properties 
-    file and authority-providers.xml for configuration details.
-    
-    Available roles:
-        ROLE_MONITOR        - for users - read only access to flow
-        ROLE_DFM            - for users - can build and configure data flows
-        ROLE_PROVENANCE     - for users - can access data flow provenance
-        ROLE_ADMIN          - for users - read only access to flow; modify user access; can purge flow configuration history
-        ROLE_PROXY          - for systems - can proxy requests on behalf of users
-        ROLE_NIFI           - for systems - can perform site to site
--->
-<users>
-    <!--
-    <user dn="[user dn - read only]">
-        <role name="ROLE_MONITOR"/>
-    </user>
-    <user dn="[user dn - data flow manager]">
-        <role name="ROLE_DFM"/>
-    </user>
-    <user dn="[user dn - read only and admin]">
-        <role name="ROLE_ADMIN"/>
-    </user>
-    <user dn="[user dn - data flow manager and admin]">
-        <role name="ROLE_DFM"/>
-        <role name="ROLE_ADMIN"/>
-    </user>
-    <user dn="[user dn - read only and provenance details]">
-        <role name="ROLE_MONITOR"/>
-        <role name="ROLE_PROVENANCE"/>
-    </user>
-    <user dn="[user dn - data flow manager and provenance details]">
-        <role name="ROLE_DFM"/>
-        <role name="ROLE_PROVENANCE"/>
-    </user>
-    <user dn="[system dn - remote NiFi performing site to site]">
-        <role name="ROLE_NIFI"/>
-    </user>
-    -->
-</users>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/authorizers.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/authorizers.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/authorizers.xml
new file mode 100644
index 0000000..01ccd9c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/authorizers.xml
@@ -0,0 +1,28 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+      http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<!--
+    This file lists the authority providers to use when running securely. In order
+    to use a specific provider it must be configured here and it's identifier
+    must be specified in the nifi.properties file.
+-->
+<authorizers>
+    <provider>
+        <identifier>file-provider</identifier>
+        <class>org.apache.nifi.authorization.FileAuthorizer</class>
+        <property name="Authorizations File">./conf/authorizations.xml</property>
+        <property name="Reload Interval">30 secs</property>
+    </provider>
+</authorizers>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
index beb71c1..f7912a1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
@@ -24,7 +24,7 @@ nifi.administrative.yield.duration=${nifi.administrative.yield.duration}
 # If a component has no work to do (is "bored"), how long should we wait before checking again for work?
 nifi.bored.yield.duration=${nifi.bored.yield.duration}
 
-nifi.authority.provider.configuration.file=${nifi.authority.provider.configuration.file}
+nifi.authorizer.configuration.file=${nifi.authorizer.configuration.file}
 nifi.login.identity.provider.configuration.file=${nifi.login.identity.provider.configuration.file}
 nifi.templates.directory=${nifi.templates.directory}
 nifi.ui.banner.text=${nifi.ui.banner.text}
@@ -137,12 +137,8 @@ nifi.security.truststore=${nifi.security.truststore}
 nifi.security.truststoreType=${nifi.security.truststoreType}
 nifi.security.truststorePasswd=${nifi.security.truststorePasswd}
 nifi.security.needClientAuth=${nifi.security.needClientAuth}
-nifi.security.user.credential.cache.duration=${nifi.security.user.credential.cache.duration}
-nifi.security.user.authority.provider=${nifi.security.user.authority.provider}
+nifi.security.user.authorizer=${nifi.security.user.authorizer}
 nifi.security.user.login.identity.provider=${nifi.security.user.login.identity.provider}
-nifi.security.support.new.account.requests=${nifi.security.support.new.account.requests}
-# Valid Authorities include: ROLE_MONITOR,ROLE_DFM,ROLE_ADMIN,ROLE_PROVENANCE,ROLE_NIFI
-nifi.security.anonymous.authorities=${nifi.security.anonymous.authorities}
 nifi.security.ocsp.responder.url=${nifi.security.ocsp.responder.url}
 nifi.security.ocsp.responder.certificate=${nifi.security.ocsp.responder.certificate}
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.java
index 66fd303..2e5f175 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.java
@@ -16,30 +16,7 @@
  */
 package org.apache.nifi.remote;
 
-import static java.util.Objects.requireNonNull;
-
-import java.io.IOException;
-import java.net.SocketTimeoutException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-
-import org.apache.nifi.admin.service.AccountDisabledException;
-import org.apache.nifi.admin.service.AccountNotFoundException;
-import org.apache.nifi.admin.service.AccountPendingException;
-import org.apache.nifi.admin.service.AdministrationException;
-import org.apache.nifi.admin.service.UserService;
-import org.apache.nifi.authorization.Authority;
+import org.apache.nifi.admin.service.KeyService;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.connectable.ConnectableType;
 import org.apache.nifi.controller.AbstractPort;
@@ -64,10 +41,27 @@ import org.apache.nifi.reporting.BulletinRepository;
 import org.apache.nifi.reporting.ComponentType;
 import org.apache.nifi.reporting.Severity;
 import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.user.NiFiUser;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
+import java.net.SocketTimeoutException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import static java.util.Objects.requireNonNull;
+
 public class StandardRootGroupPort extends AbstractPort implements RootGroupPort {
 
     private static final String CATEGORY = "Site to Site";
@@ -78,7 +72,7 @@ public class StandardRootGroupPort extends AbstractPort implements RootGroupPort
     private final AtomicReference<Set<String>> userAccessControl = new AtomicReference<Set<String>>(new HashSet<String>());
     private final ProcessScheduler processScheduler;
     private final boolean secure;
-    private final UserService userService;
+    private final KeyService keyService;
     @SuppressWarnings("unused")
     private final BulletinRepository bulletinRepository;
     private final EventReporter eventReporter;
@@ -92,13 +86,13 @@ public class StandardRootGroupPort extends AbstractPort implements RootGroupPort
     private boolean shutdown = false;   // guarded by requestLock
 
     public StandardRootGroupPort(final String id, final String name, final ProcessGroup processGroup,
-            final TransferDirection direction, final ConnectableType type, final UserService userService,
+            final TransferDirection direction, final ConnectableType type, final KeyService keyService,
             final BulletinRepository bulletinRepository, final ProcessScheduler scheduler, final boolean secure) {
         super(id, name, processGroup, type, scheduler);
 
         this.processScheduler = scheduler;
         setScheduldingPeriod(MINIMUM_SCHEDULING_NANOS + " nanos");
-        this.userService = userService;
+        this.keyService = keyService;
         this.secure = secure;
         this.bulletinRepository = bulletinRepository;
         this.scheduler = scheduler;
@@ -355,67 +349,8 @@ public class StandardRootGroupPort extends AbstractPort implements RootGroupPort
             return new StandardPortAuthorizationResult(false, "User DN is not known");
         }
 
-        try {
-            final NiFiUser user = userService.checkAuthorization(dn);
-
-            final Set<Authority> authorities = user.getAuthorities();
-            if (!authorities.contains(Authority.ROLE_NIFI)) {
-                final String message = String.format("%s authorization failed for user %s because the user does not have Role NiFi", this, dn);
-                logger.warn(message);
-                eventReporter.reportEvent(Severity.WARNING, CATEGORY, message);
-                return new StandardPortAuthorizationResult(false, "User does not contain required Role: NiFi");
-            }
-
-            final Set<String> allowedUsers = userAccessControl.get();
-            if (allowedUsers.contains(dn)) {
-                return new StandardPortAuthorizationResult(true, "User is Authorized");
-            }
-
-            final String userGroup = user.getUserGroup();
-            if (userGroup == null) {
-                final String message = String.format("%s authorization failed for user %s because the user does not have a group and is not in the set of Allowed Users for this Port", this, dn);
-                logger.warn(message);
-                eventReporter.reportEvent(Severity.WARNING, CATEGORY, message);
-                return new StandardPortAuthorizationResult(false, "User is not Authorized to communicate with " + this.toString());
-            }
-
-            final Set<String> allowedGroups = groupAccessControl.get();
-            final boolean allowed = allowedGroups.contains(userGroup);
-            if (!allowed) {
-                final String message = String.format("%s authorization failed for user %s because the user "
-                        + "is not in the set of Allowed Users, and the user's group is not in the set of Allowed Groups for this Port", this, dn);
-                logger.warn(message);
-                eventReporter.reportEvent(Severity.WARNING, CATEGORY, message);
-                return new StandardPortAuthorizationResult(false, "User is not Authorized to communicate with " + this.toString());
-            }
-
-            return new StandardPortAuthorizationResult(true, "User is part of group '" + userGroup + "', which is Authorized to communicate with " + this.toString());
-        } catch (final AccountNotFoundException anfe) {
-            final String message = String.format("%s authorization failed for user %s because the DN is unknown", this, dn);
-            logger.warn(message);
-            eventReporter.reportEvent(Severity.WARNING, CATEGORY, message);
-            return new StandardPortAuthorizationResult(false, "User DN is not known");
-        } catch (final AccountDisabledException ade) {
-            final String message = String.format("%s authorization failed for user %s because the User Status is not 'ACTIVE' but instead is 'DISABLED'", this, dn);
-            logger.warn(message);
-            eventReporter.reportEvent(Severity.WARNING, CATEGORY, message);
-            return new StandardPortAuthorizationResult(false, "User Status is 'DISABLED' rather than 'ACTIVE'");
-        } catch (final AccountPendingException ape) {
-            final String message = String.format("%s authorization failed for user %s because the User Status is not 'ACTIVE' but instead is 'PENDING'", this, dn);
-            logger.warn(message);
-            eventReporter.reportEvent(Severity.WARNING, CATEGORY, message);
-            return new StandardPortAuthorizationResult(false, "User Status is 'PENDING' rather than 'ACTIVE'");
-        } catch (final AdministrationException ae) {
-            final String message = String.format("%s authorization failed for user %s because ", this, dn, ae);
-            logger.warn(message);
-            eventReporter.reportEvent(Severity.WARNING, CATEGORY, message);
-            return new StandardPortAuthorizationResult(false, "Authorization failed because " + ae);
-        } catch (final Exception e) {
-            final String message = String.format("%s authorization failed for user %s because ", this, dn, e);
-            logger.warn(message);
-            eventReporter.reportEvent(Severity.WARNING, CATEGORY, message);
-            return new StandardPortAuthorizationResult(false, "Authorization failed because " + e);
-        }
+        // TODO - Replace with call to Authorizer to authorize site to site data transfer
+        return new StandardPortAuthorizationResult(true, "User is Authorized");
     }
 
     public static class StandardPortAuthorizationResult implements PortAuthorizationResult {

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/resources/nifi.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/resources/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/resources/nifi.properties
index 4364bff..5b1134b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/resources/nifi.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/resources/nifi.properties
@@ -63,11 +63,7 @@ nifi.security.truststore=src/test/resources/dummy-certs/localhost-ts.jks
 nifi.security.truststoreType=JKS
 nifi.security.truststorePasswd=localtest
 nifi.security.needClientAuth=true
-nifi.security.authorizedUsers.file=./conf/authorized-users.xml
-nifi.security.user.credential.cache.duration.seconds=
-nifi.security.user.authority.provider=nifi.cluster.authorization.ClusterAuthorizationProvider
-nifi.security.support.new.account.requests=
-nifi.security.default.user.roles=
+nifi.security.user.authorizer=
 
 # cluster common properties (cluster manager and nodes must have same values) #
 nifi.cluster.protocol.heartbeat.tick.seconds=10

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/FunnelAuditor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/FunnelAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/FunnelAuditor.java
index 3949028..40f0f34 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/FunnelAuditor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/FunnelAuditor.java
@@ -67,18 +67,17 @@ public class FunnelAuditor extends NiFiAuditor {
      * Audits the removal of a funnel.
      *
      * @param proceedingJoinPoint join point
-     * @param groupId group id
      * @param funnelId funnel id
      * @param funnelDAO funnel dao
      * @throws Throwable ex
      */
     @Around("within(org.apache.nifi.web.dao.FunnelDAO+) && "
-            + "execution(void deleteFunnel(java.lang.String, java.lang.String)) && "
-            + "args(groupId, funnelId) && "
+            + "execution(void deleteFunnel(java.lang.String)) && "
+            + "args(funnelId) && "
             + "target(funnelDAO)")
-    public void removeFunnelAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, String funnelId, FunnelDAO funnelDAO) throws Throwable {
+    public void removeFunnelAdvice(ProceedingJoinPoint proceedingJoinPoint, String funnelId, FunnelDAO funnelDAO) throws Throwable {
         // get the funnel before removing it
-        Funnel funnel = funnelDAO.getFunnel(groupId, funnelId);
+        Funnel funnel = funnelDAO.getFunnel(funnelId);
 
         // remove the funnel
         proceedingJoinPoint.proceed();

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/PortAuditor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/PortAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/PortAuditor.java
index e99a1aa..dff311e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/PortAuditor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/PortAuditor.java
@@ -16,12 +16,7 @@
  */
 package org.apache.nifi.audit;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Date;
-import java.util.HashSet;
-import java.util.Set;
-
+import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.action.Action;
 import org.apache.nifi.action.Component;
 import org.apache.nifi.action.FlowChangeAction;
@@ -32,18 +27,22 @@ import org.apache.nifi.connectable.ConnectableType;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.remote.RootGroupPort;
-import org.apache.nifi.web.security.user.NiFiUserUtils;
 import org.apache.nifi.user.NiFiUser;
 import org.apache.nifi.web.api.dto.PortDTO;
 import org.apache.nifi.web.dao.PortDAO;
-
-import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.web.security.user.NiFiUserUtils;
 import org.aspectj.lang.ProceedingJoinPoint;
 import org.aspectj.lang.annotation.Around;
 import org.aspectj.lang.annotation.Aspect;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Set;
+
 @Aspect
 public class PortAuditor extends NiFiAuditor {
 
@@ -77,18 +76,17 @@ public class PortAuditor extends NiFiAuditor {
      * Audits the update of a port.
      *
      * @param proceedingJoinPoint join point
-     * @param groupId group id
      * @param portDTO port dto
      * @param portDAO port dao
      * @return port
      * @throws Throwable ex
      */
     @Around("within(org.apache.nifi.web.dao.PortDAO+) && "
-            + "execution(org.apache.nifi.connectable.Port updatePort(java.lang.String, org.apache.nifi.web.api.dto.PortDTO)) && "
-            + "args(groupId, portDTO) && "
+            + "execution(org.apache.nifi.connectable.Port updatePort(org.apache.nifi.web.api.dto.PortDTO)) && "
+            + "args(portDTO) && "
             + "target(portDAO)")
-    public Port updatePortAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, PortDTO portDTO, PortDAO portDAO) throws Throwable {
-        final Port port = portDAO.getPort(groupId, portDTO.getId());
+    public Port updatePortAdvice(ProceedingJoinPoint proceedingJoinPoint, PortDTO portDTO, PortDAO portDAO) throws Throwable {
+        final Port port = portDAO.getPort(portDTO.getId());
         final ScheduledState scheduledState = port.getScheduledState();
         final String name = port.getName();
         final String comments = port.getComments();
@@ -262,18 +260,17 @@ public class PortAuditor extends NiFiAuditor {
      * Audits the removal of a processor via deleteProcessor().
      *
      * @param proceedingJoinPoint join point
-     * @param groupId group id
      * @param portId port id
      * @param portDAO port dao
      * @throws Throwable ex
      */
     @Around("within(org.apache.nifi.web.dao.PortDAO+) && "
-            + "execution(void deletePort(java.lang.String, java.lang.String)) && "
-            + "args(groupId, portId) && "
+            + "execution(void deletePort(java.lang.String)) && "
+            + "args(portId) && "
             + "target(portDAO)")
-    public void removePortAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, String portId, PortDAO portDAO) throws Throwable {
+    public void removePortAdvice(ProceedingJoinPoint proceedingJoinPoint, String portId, PortDAO portDAO) throws Throwable {
         // get the port before removing it
-        Port port = portDAO.getPort(groupId, portId);
+        Port port = portDAO.getPort(portId);
 
         // remove the port
         proceedingJoinPoint.proceed();

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java
index 4f147fb..d6bf700 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java
@@ -100,19 +100,18 @@ public class ProcessorAuditor extends NiFiAuditor {
      * Audits the configuration of a single processor.
      *
      * @param proceedingJoinPoint join point
-     * @param groupId group id
      * @param processorDTO dto
      * @param processorDAO dao
      * @return node
      * @throws Throwable ex
      */
     @Around("within(org.apache.nifi.web.dao.ProcessorDAO+) && "
-            + "execution(org.apache.nifi.controller.ProcessorNode updateProcessor(java.lang.String, org.apache.nifi.web.api.dto.ProcessorDTO)) && "
-            + "args(groupId, processorDTO) && "
+            + "execution(org.apache.nifi.controller.ProcessorNode updateProcessor(org.apache.nifi.web.api.dto.ProcessorDTO)) && "
+            + "args(processorDTO) && "
             + "target(processorDAO)")
-    public ProcessorNode updateProcessorAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, ProcessorDTO processorDTO, ProcessorDAO processorDAO) throws Throwable {
+    public ProcessorNode updateProcessorAdvice(ProceedingJoinPoint proceedingJoinPoint, ProcessorDTO processorDTO, ProcessorDAO processorDAO) throws Throwable {
         // determine the initial values for each property/setting thats changing
-        ProcessorNode processor = processorDAO.getProcessor(groupId, processorDTO.getId());
+        ProcessorNode processor = processorDAO.getProcessor(processorDTO.getId());
         final Map<String, String> values = extractConfiguredPropertyValues(processor, processorDTO);
         final ScheduledState scheduledState = processor.getScheduledState();
 
@@ -121,7 +120,7 @@ public class ProcessorAuditor extends NiFiAuditor {
 
         // if no exceptions were thrown, add the processor action...
         // get the updated verbose state
-        processor = processorDAO.getProcessor(updatedProcessor.getProcessGroup().getIdentifier(), updatedProcessor.getIdentifier());
+        processor = processorDAO.getProcessor(updatedProcessor.getIdentifier());
 
         // get the current user
         NiFiUser user = NiFiUserUtils.getNiFiUser();
@@ -235,18 +234,17 @@ public class ProcessorAuditor extends NiFiAuditor {
      * Audits the removal of a processor via deleteProcessor().
      *
      * @param proceedingJoinPoint join point
-     * @param groupId group id
      * @param processorId processor id
      * @param processorDAO dao
      * @throws Throwable ex
      */
     @Around("within(org.apache.nifi.web.dao.ProcessorDAO+) && "
-            + "execution(void deleteProcessor(java.lang.String, java.lang.String)) && "
-            + "args(groupId, processorId) && "
+            + "execution(void deleteProcessor(java.lang.String)) && "
+            + "args(processorId) && "
             + "target(processorDAO)")
-    public void removeProcessorAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, String processorId, ProcessorDAO processorDAO) throws Throwable {
+    public void removeProcessorAdvice(ProceedingJoinPoint proceedingJoinPoint, String processorId, ProcessorDAO processorDAO) throws Throwable {
         // get the processor before removing it
-        ProcessorNode processor = processorDAO.getProcessor(groupId, processorId);
+        ProcessorNode processor = processorDAO.getProcessor(processorId);
 
         // remove the processor
         proceedingJoinPoint.proceed();

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RelationshipAuditor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RelationshipAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RelationshipAuditor.java
index 95000d8..8a77636 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RelationshipAuditor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RelationshipAuditor.java
@@ -100,19 +100,18 @@ public class RelationshipAuditor extends NiFiAuditor {
      * Audits the creation and removal of relationships via updateConnection().
      *
      * @param proceedingJoinPoint join point
-     * @param groupId group id
      * @param connectionDTO dto
      * @param connectionDAO dao
      * @return connection
      * @throws Throwable ex
      */
     @Around("within(org.apache.nifi.web.dao.ConnectionDAO+) && "
-            + "execution(org.apache.nifi.connectable.Connection updateConnection(java.lang.String, org.apache.nifi.web.api.dto.ConnectionDTO)) && "
-            + "args(groupId, connectionDTO) && "
+            + "execution(org.apache.nifi.connectable.Connection updateConnection(org.apache.nifi.web.api.dto.ConnectionDTO)) && "
+            + "args(connectionDTO) && "
             + "target(connectionDAO)")
-    public Connection updateConnectionAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, ConnectionDTO connectionDTO, ConnectionDAO connectionDAO) throws Throwable {
+    public Connection updateConnectionAdvice(ProceedingJoinPoint proceedingJoinPoint, ConnectionDTO connectionDTO, ConnectionDAO connectionDAO) throws Throwable {
         // get the previous configuration
-        Connection connection = connectionDAO.getConnection(groupId, connectionDTO.getId());
+        Connection connection = connectionDAO.getConnection(connectionDTO.getId());
         Connectable previousDestination = connection.getDestination();
         Collection<Relationship> previousRelationships = connection.getRelationships();
         Map<String, String> values = extractConfiguredPropertyValues(connection, connectionDTO);
@@ -214,18 +213,17 @@ public class RelationshipAuditor extends NiFiAuditor {
      * Audits the removal of relationships via deleteConnection().
      *
      * @param proceedingJoinPoint join point
-     * @param groupId group id
      * @param id id
      * @param connectionDAO dao
      * @throws Throwable ex
      */
     @Around("within(org.apache.nifi.web.dao.ConnectionDAO+) && "
-            + "execution(void deleteConnection(java.lang.String, java.lang.String)) && "
-            + "args(groupId, id) && "
+            + "execution(void deleteConnection(java.lang.String)) && "
+            + "args(id) && "
             + "target(connectionDAO)")
-    public void removeConnectionAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, String id, ConnectionDAO connectionDAO) throws Throwable {
+    public void removeConnectionAdvice(ProceedingJoinPoint proceedingJoinPoint, String id, ConnectionDAO connectionDAO) throws Throwable {
         // get the connection before performing the update
-        Connection connection = connectionDAO.getConnection(groupId, id);
+        Connection connection = connectionDAO.getConnection(id);
 
         // perform the underlying operation
         proceedingJoinPoint.proceed();

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RemoteProcessGroupAuditor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RemoteProcessGroupAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RemoteProcessGroupAuditor.java
index 5815634..0495e99 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RemoteProcessGroupAuditor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RemoteProcessGroupAuditor.java
@@ -16,11 +16,6 @@
  */
 package org.apache.nifi.audit;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.Map;
 import org.apache.nifi.action.Action;
 import org.apache.nifi.action.Component;
 import org.apache.nifi.action.FlowChangeAction;
@@ -30,18 +25,24 @@ import org.apache.nifi.action.details.ActionDetails;
 import org.apache.nifi.action.details.FlowChangeConfigureDetails;
 import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.remote.RemoteGroupPort;
-import org.apache.nifi.web.security.user.NiFiUserUtils;
 import org.apache.nifi.user.NiFiUser;
 import org.apache.nifi.web.api.dto.RemoteProcessGroupContentsDTO;
 import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
 import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
 import org.apache.nifi.web.dao.RemoteProcessGroupDAO;
+import org.apache.nifi.web.security.user.NiFiUserUtils;
 import org.aspectj.lang.ProceedingJoinPoint;
 import org.aspectj.lang.annotation.Around;
 import org.aspectj.lang.annotation.Aspect;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+
 /**
  * Audits remote process group creation/removal and configuration changes.
  */
@@ -82,19 +83,18 @@ public class RemoteProcessGroupAuditor extends NiFiAuditor {
      * Audits the update of remote process group configuration.
      *
      * @param proceedingJoinPoint join point
-     * @param groupId group id
      * @param remoteProcessGroupDTO dto
      * @param remoteProcessGroupDAO dao
      * @return group
      * @throws Throwable ex
      */
     @Around("within(org.apache.nifi.web.dao.RemoteProcessGroupDAO+) && "
-            + "execution(org.apache.nifi.groups.RemoteProcessGroup updateRemoteProcessGroup(java.lang.String, org.apache.nifi.web.api.dto.RemoteProcessGroupDTO)) && "
-            + "args(groupId, remoteProcessGroupDTO) && "
+            + "execution(org.apache.nifi.groups.RemoteProcessGroup updateRemoteProcessGroup(org.apache.nifi.web.api.dto.RemoteProcessGroupDTO)) && "
+            + "args(remoteProcessGroupDTO) && "
             + "target(remoteProcessGroupDAO)")
     public RemoteProcessGroup auditUpdateProcessGroupConfiguration(
-            ProceedingJoinPoint proceedingJoinPoint, String groupId, RemoteProcessGroupDTO remoteProcessGroupDTO, RemoteProcessGroupDAO remoteProcessGroupDAO) throws Throwable {
-        final RemoteProcessGroup remoteProcessGroup = remoteProcessGroupDAO.getRemoteProcessGroup(groupId, remoteProcessGroupDTO.getId());
+            ProceedingJoinPoint proceedingJoinPoint, RemoteProcessGroupDTO remoteProcessGroupDTO, RemoteProcessGroupDAO remoteProcessGroupDAO) throws Throwable {
+        final RemoteProcessGroup remoteProcessGroup = remoteProcessGroupDAO.getRemoteProcessGroup(remoteProcessGroupDTO.getId());
 
         // record the current value of this remoteProcessGroups configuration for comparisons later
         final boolean transmissionState = remoteProcessGroup.isTransmitting();
@@ -298,18 +298,17 @@ public class RemoteProcessGroupAuditor extends NiFiAuditor {
      * Audits the removal of a process group via deleteProcessGroup().
      *
      * @param proceedingJoinPoint join point
-     * @param groupId group id
      * @param remoteProcessGroupId remote group id
      * @param remoteProcessGroupDAO remote group dao
      * @throws Throwable ex
      */
     @Around("within(org.apache.nifi.web.dao.RemoteProcessGroupDAO+) && "
-            + "execution(void deleteRemoteProcessGroup(java.lang.String, java.lang.String)) && "
-            + "args(groupId, remoteProcessGroupId) && "
+            + "execution(void deleteRemoteProcessGroup(java.lang.String)) && "
+            + "args(remoteProcessGroupId) && "
             + "target(remoteProcessGroupDAO)")
-    public void removeRemoteProcessGroupAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, String remoteProcessGroupId, RemoteProcessGroupDAO remoteProcessGroupDAO) throws Throwable {
+    public void removeRemoteProcessGroupAdvice(ProceedingJoinPoint proceedingJoinPoint, String remoteProcessGroupId, RemoteProcessGroupDAO remoteProcessGroupDAO) throws Throwable {
         // get the remote process group before removing it
-        RemoteProcessGroup remoteProcessGroup = remoteProcessGroupDAO.getRemoteProcessGroup(groupId, remoteProcessGroupId);
+        RemoteProcessGroup remoteProcessGroup = remoteProcessGroupDAO.getRemoteProcessGroup(remoteProcessGroupId);
 
         // remove the remote process group
         proceedingJoinPoint.proceed();

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/SnippetAuditor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/SnippetAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/SnippetAuditor.java
index 4b7c38a..3fcc419 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/SnippetAuditor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/SnippetAuditor.java
@@ -16,12 +16,8 @@
  */
 package org.apache.nifi.audit;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Date;
-import java.util.HashSet;
-import java.util.Set;
-
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.action.Action;
 import org.apache.nifi.action.Component;
 import org.apache.nifi.action.FlowChangeAction;
@@ -38,7 +34,6 @@ import org.apache.nifi.controller.ProcessorNode;
 import org.apache.nifi.controller.Snippet;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.web.security.user.NiFiUserUtils;
 import org.apache.nifi.user.NiFiUser;
 import org.apache.nifi.web.api.dto.ConnectableDTO;
 import org.apache.nifi.web.api.dto.ConnectionDTO;
@@ -56,15 +51,19 @@ import org.apache.nifi.web.dao.ProcessGroupDAO;
 import org.apache.nifi.web.dao.ProcessorDAO;
 import org.apache.nifi.web.dao.RemoteProcessGroupDAO;
 import org.apache.nifi.web.dao.SnippetDAO;
-
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.web.security.user.NiFiUserUtils;
 import org.aspectj.lang.ProceedingJoinPoint;
 import org.aspectj.lang.annotation.Around;
 import org.aspectj.lang.annotation.Aspect;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Set;
+
 /**
  *
  */
@@ -273,7 +272,7 @@ public class SnippetAuditor extends NiFiAuditor {
             final Collection<Action> actions = new ArrayList<>();
 
             for (String id : snippet.getProcessors()) {
-                final ProcessorNode processor = processorDAO.getProcessor(groupId, id);
+                final ProcessorNode processor = processorDAO.getProcessor(id);
                 final Action action = processorAuditor.generateAuditRecord(processor, Operation.Move, createMoveDetails(previousGroupId, groupId, logger));
                 if (action != null) {
                     actions.add(action);
@@ -281,7 +280,7 @@ public class SnippetAuditor extends NiFiAuditor {
             }
 
             for (String id : snippet.getFunnels()) {
-                final Funnel funnel = funnelDAO.getFunnel(groupId, id);
+                final Funnel funnel = funnelDAO.getFunnel(id);
                 final Action action = funnelAuditor.generateAuditRecord(funnel, Operation.Move, createMoveDetails(previousGroupId, groupId, logger));
                 if (action != null) {
                     actions.add(action);
@@ -289,7 +288,7 @@ public class SnippetAuditor extends NiFiAuditor {
             }
 
             for (String id : snippet.getInputPorts()) {
-                final Port port = inputPortDAO.getPort(groupId, id);
+                final Port port = inputPortDAO.getPort(id);
                 final Action action = portAuditor.generateAuditRecord(port, Operation.Move, createMoveDetails(previousGroupId, groupId, logger));
                 if (action != null) {
                     actions.add(action);
@@ -297,7 +296,7 @@ public class SnippetAuditor extends NiFiAuditor {
             }
 
             for (String id : snippet.getOutputPorts()) {
-                final Port port = outputPortDAO.getPort(groupId, id);
+                final Port port = outputPortDAO.getPort(id);
                 final Action action = portAuditor.generateAuditRecord(port, Operation.Move, createMoveDetails(previousGroupId, groupId, logger));
                 if (action != null) {
                     actions.add(action);
@@ -305,7 +304,7 @@ public class SnippetAuditor extends NiFiAuditor {
             }
 
             for (String id : snippet.getRemoteProcessGroups()) {
-                final RemoteProcessGroup remoteProcessGroup = remoteProcessGroupDAO.getRemoteProcessGroup(groupId, id);
+                final RemoteProcessGroup remoteProcessGroup = remoteProcessGroupDAO.getRemoteProcessGroup(id);
                 final Action action = remoteProcessGroupAuditor.generateAuditRecord(remoteProcessGroup, Operation.Move, createMoveDetails(previousGroupId, groupId, logger));
                 if (action != null) {
                     actions.add(action);
@@ -322,7 +321,7 @@ public class SnippetAuditor extends NiFiAuditor {
             }
 
             for (String id : snippet.getConnections()) {
-                final Connection connection = connectionDAO.getConnection(groupId, id);
+                final Connection connection = connectionDAO.getConnection(id);
                 final Action action = relationshipAuditor.generateAuditRecordForConnection(connection, Operation.Move, createMoveDetails(previousGroupId, groupId, logger));
                 if (action != null) {
                     actions.add(action);
@@ -355,27 +354,25 @@ public class SnippetAuditor extends NiFiAuditor {
         final Snippet snippet = snippetDAO.getSnippet(snippetId);
 
         if (snippet.isLinked()) {
-            final String groupId = snippet.getParentGroupId();
-
             // locate all the components being removed
             final Set<Funnel> funnels = new HashSet<>();
             for (String id : snippet.getFunnels()) {
-                funnels.add(funnelDAO.getFunnel(groupId, id));
+                funnels.add(funnelDAO.getFunnel(id));
             }
 
             final Set<Port> inputPorts = new HashSet<>();
             for (String id : snippet.getInputPorts()) {
-                inputPorts.add(inputPortDAO.getPort(groupId, id));
+                inputPorts.add(inputPortDAO.getPort(id));
             }
 
             final Set<Port> outputPorts = new HashSet<>();
             for (String id : snippet.getOutputPorts()) {
-                outputPorts.add(outputPortDAO.getPort(groupId, id));
+                outputPorts.add(outputPortDAO.getPort(id));
             }
 
             final Set<RemoteProcessGroup> remoteProcessGroups = new HashSet<>();
             for (String id : snippet.getRemoteProcessGroups()) {
-                remoteProcessGroups.add(remoteProcessGroupDAO.getRemoteProcessGroup(groupId, id));
+                remoteProcessGroups.add(remoteProcessGroupDAO.getRemoteProcessGroup(id));
             }
 
             final Set<ProcessGroup> processGroups = new HashSet<>();
@@ -386,12 +383,12 @@ public class SnippetAuditor extends NiFiAuditor {
 
             final Set<ProcessorNode> processors = new HashSet<>();
             for (String id : snippet.getProcessors()) {
-                processors.add(processorDAO.getProcessor(groupId, id));
+                processors.add(processorDAO.getProcessor(id));
             }
 
             final Set<Connection> connections = new HashSet<>();
             for (String id : snippet.getConnections()) {
-                connections.add(connectionDAO.getConnection(groupId, id));
+                connections.add(connectionDAO.getConnection(id));
             }
 
             // remove the snippet and components