You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by mc...@apache.org on 2016/04/07 22:23:32 UTC

[4/9] nifi git commit: Revert "NIFI-1551:"

http://git-wip-us.apache.org/repos/asf/nifi/blob/3f4ac315/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/src/main/java/org/apache/nifi/authorization/FileAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/src/main/java/org/apache/nifi/authorization/FileAuthorizationProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/src/main/java/org/apache/nifi/authorization/FileAuthorizationProvider.java
new file mode 100644
index 0000000..9c2cad5
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/src/main/java/org/apache/nifi/authorization/FileAuthorizationProvider.java
@@ -0,0 +1,496 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.authorization;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.xml.XMLConstants;
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBElement;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Marshaller;
+import javax.xml.bind.Unmarshaller;
+import javax.xml.transform.stream.StreamSource;
+import javax.xml.validation.Schema;
+import javax.xml.validation.SchemaFactory;
+import org.apache.nifi.authorization.annotation.AuthorityProviderContext;
+import org.apache.nifi.authorization.exception.AuthorityAccessException;
+import org.apache.nifi.authorization.exception.IdentityAlreadyExistsException;
+import org.apache.nifi.authorization.exception.ProviderCreationException;
+import org.apache.nifi.authorization.exception.UnknownIdentityException;
+import org.apache.nifi.util.file.FileUtils;
+import org.apache.nifi.user.generated.ObjectFactory;
+import org.apache.nifi.user.generated.Role;
+import org.apache.nifi.user.generated.User;
+import org.apache.nifi.user.generated.Users;
+import org.apache.nifi.util.NiFiProperties;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.xml.sax.SAXException;
+
+/**
+ * Provides identity checks and grants authorities.
+ */
+public class FileAuthorizationProvider implements AuthorityProvider {
+
+    private static final Logger logger = LoggerFactory.getLogger(FileAuthorizationProvider.class);
+    private static final String USERS_XSD = "/users.xsd";
+    private static final String JAXB_GENERATED_PATH = "org.apache.nifi.user.generated";
+    private static final JAXBContext JAXB_CONTEXT = initializeJaxbContext();
+
+    /**
+     * Load the JAXBContext.
+     */
+    private static JAXBContext initializeJaxbContext() {
+        try {
+            return JAXBContext.newInstance(JAXB_GENERATED_PATH, FileAuthorizationProvider.class.getClassLoader());
+        } catch (JAXBException e) {
+            throw new RuntimeException("Unable to create JAXBContext.");
+        }
+    }
+
+    private NiFiProperties properties;
+    private File usersFile;
+    private File restoreUsersFile;
+    private Users users;
+    private final Set<String> defaultAuthorities = new HashSet<>();
+
+    @Override
+    public void initialize(final AuthorityProviderInitializationContext initializationContext) throws ProviderCreationException {
+    }
+
+    @Override
+    public void onConfigured(final AuthorityProviderConfigurationContext configurationContext) throws ProviderCreationException {
+        try {
+            final String usersFilePath = configurationContext.getProperty("Authorized Users File");
+            if (usersFilePath == null || usersFilePath.trim().isEmpty()) {
+                throw new ProviderCreationException("The authorized users file must be specified.");
+            }
+
+            // the users file instance will never be null because a default is used
+            usersFile = new File(usersFilePath);
+            final File usersFileDirectory = usersFile.getParentFile();
+
+            // the restore directory is optional and may be null
+            final File restoreDirectory = properties.getRestoreDirectory();
+
+            if (restoreDirectory != null) {
+
+                // sanity check that restore directory is a directory, creating it if necessary
+                FileUtils.ensureDirectoryExistAndCanAccess(restoreDirectory);
+
+                // check that restore directory is not the same as the primary directory
+                if (usersFileDirectory.getAbsolutePath().equals(restoreDirectory.getAbsolutePath())) {
+                    throw new ProviderCreationException(String.format("Authorized User's directory '%s' is the same as restore directory '%s' ",
+                            usersFileDirectory.getAbsolutePath(), restoreDirectory.getAbsolutePath()));
+                }
+
+                // the restore copy will have same file name, but reside in a different directory
+                restoreUsersFile = new File(restoreDirectory, usersFile.getName());
+
+                // sync the primary copy with the restore copy
+                try {
+                    FileUtils.syncWithRestore(usersFile, restoreUsersFile, logger);
+                } catch (final IOException | IllegalStateException ioe) {
+                    throw new ProviderCreationException(ioe);
+                }
+
+            }
+
+            // load the users from the specified file
+            if (usersFile.exists()) {
+                // find the schema
+                final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
+                final Schema schema = schemaFactory.newSchema(FileAuthorizationProvider.class.getResource(USERS_XSD));
+
+                // attempt to unmarshal
+                final Unmarshaller unmarshaller = JAXB_CONTEXT.createUnmarshaller();
+                unmarshaller.setSchema(schema);
+                final JAXBElement<Users> element = unmarshaller.unmarshal(new StreamSource(usersFile), Users.class);
+                users = element.getValue();
+            } else {
+                final ObjectFactory objFactory = new ObjectFactory();
+                users = objFactory.createUsers();
+            }
+
+            // attempt to load a default roles
+            final String rawDefaultAuthorities = configurationContext.getProperty("Default User Roles");
+            if (StringUtils.isNotBlank(rawDefaultAuthorities)) {
+                final Set<String> invalidDefaultAuthorities = new HashSet<>();
+
+                // validate the specified authorities
+                final String[] rawDefaultAuthorityList = rawDefaultAuthorities.split(",");
+                for (String rawAuthority : rawDefaultAuthorityList) {
+                    rawAuthority = rawAuthority.trim();
+                    final Authority authority = Authority.valueOfAuthority(rawAuthority);
+                    if (authority == null) {
+                        invalidDefaultAuthorities.add(rawAuthority);
+                    } else {
+                        defaultAuthorities.add(rawAuthority);
+                    }
+                }
+
+                // report any unrecognized authorities
+                if (!invalidDefaultAuthorities.isEmpty()) {
+                    logger.warn(String.format("The following default role(s) '%s' were not recognized. Possible values: %s.",
+                            StringUtils.join(invalidDefaultAuthorities, ", "), StringUtils.join(Authority.getRawAuthorities(), ", ")));
+                }
+            }
+        } catch (IOException | ProviderCreationException | SAXException | JAXBException e) {
+            throw new ProviderCreationException(e);
+        }
+
+    }
+
+    @Override
+    public void preDestruction() {
+    }
+
+    private boolean hasDefaultRoles() {
+        return !defaultAuthorities.isEmpty();
+    }
+
+    @Override
+    public boolean doesDnExist(String dn) throws AuthorityAccessException {
+        if (hasDefaultRoles()) {
+            return true;
+        }
+
+        final User user = getUser(dn);
+        return user != null;
+    }
+
+    @Override
+    public synchronized Set<Authority> getAuthorities(String dn) throws UnknownIdentityException, AuthorityAccessException {
+        final Set<Authority> authorities = EnumSet.noneOf(Authority.class);
+
+        // get the user
+        final User user = getUser(dn);
+
+        // ensure the user was located
+        if (user == null) {
+            if (hasDefaultRoles()) {
+                logger.debug(String.format("User DN not found: %s. Creating new user with default roles.", dn));
+
+                // create the user (which will automatically add any default authorities)
+                addUser(dn, null);
+
+                // get the authorities for the newly created user
+                authorities.addAll(getAuthorities(dn));
+            } else {
+                throw new UnknownIdentityException(String.format("User DN not found: %s.", dn));
+            }
+        } else {
+            // create the authorities that this user has
+            for (final Role role : user.getRole()) {
+                authorities.add(Authority.valueOfAuthority(role.getName()));
+            }
+        }
+
+        return authorities;
+    }
+
+    @Override
+    public synchronized void setAuthorities(String dn, Set<Authority> authorities) throws UnknownIdentityException, AuthorityAccessException {
+        // get the user
+        final User user = getUser(dn);
+
+        // ensure the user was located
+        if (user == null) {
+            throw new UnknownIdentityException(String.format("User DN not found: %s.", dn));
+        }
+
+        // add the user authorities
+        setUserAuthorities(user, authorities);
+
+        try {
+            // save the file
+            save();
+        } catch (Exception e) {
+            throw new AuthorityAccessException(e.getMessage(), e);
+        }
+    }
+
+    private void setUserAuthorities(final User user, final Set<Authority> authorities) {
+        // clear the existing rules
+        user.getRole().clear();
+
+        // set the new roles
+        final ObjectFactory objFactory = new ObjectFactory();
+        for (final Authority authority : authorities) {
+            final Role role = objFactory.createRole();
+            role.setName(authority.toString());
+
+            // add the new role
+            user.getRole().add(role);
+        }
+    }
+
+    @Override
+    public synchronized void addUser(String dn, String group) throws IdentityAlreadyExistsException, AuthorityAccessException {
+        final User user = getUser(dn);
+
+        // ensure the user doesn't already exist
+        if (user != null) {
+            throw new IdentityAlreadyExistsException(String.format("User DN already exists: %s", dn));
+        }
+
+        // create the new user
+        final ObjectFactory objFactory = new ObjectFactory();
+        final User newUser = objFactory.createUser();
+
+        // set the user properties
+        newUser.setDn(dn);
+        newUser.setGroup(group);
+
+        // add default roles if appropriate
+        if (hasDefaultRoles()) {
+            for (final String authority : defaultAuthorities) {
+                Role role = objFactory.createRole();
+                role.setName(authority);
+
+                // add the role
+                newUser.getRole().add(role);
+            }
+        }
+
+        // add the user
+        users.getUser().add(newUser);
+
+        try {
+            // save the file
+            save();
+        } catch (Exception e) {
+            throw new AuthorityAccessException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public synchronized Set<String> getUsers(Authority authority) throws AuthorityAccessException {
+        final Set<String> userSet = new HashSet<>();
+        for (final User user : users.getUser()) {
+            for (final Role role : user.getRole()) {
+                if (role.getName().equals(authority.toString())) {
+                    userSet.add(user.getDn());
+                }
+            }
+        }
+        return userSet;
+    }
+
+    @Override
+    public synchronized void revokeUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
+        // get the user
+        final User user = getUser(dn);
+
+        // ensure the user was located
+        if (user == null) {
+            throw new UnknownIdentityException(String.format("User DN not found: %s.", dn));
+        }
+
+        // remove the specified user
+        users.getUser().remove(user);
+
+        try {
+            // save the file
+            save();
+        } catch (Exception e) {
+            throw new AuthorityAccessException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public void setUsersGroup(Set<String> dns, String group) throws UnknownIdentityException, AuthorityAccessException {
+        final Collection<User> groupedUsers = new HashSet<>();
+
+        // get the specified users
+        for (final String dn : dns) {
+            // get the user
+            final User user = getUser(dn);
+
+            // ensure the user was located
+            if (user == null) {
+                throw new UnknownIdentityException(String.format("User DN not found: %s.", dn));
+            }
+
+            groupedUsers.add(user);
+        }
+
+        // update each user group
+        for (final User user : groupedUsers) {
+            user.setGroup(group);
+        }
+
+        try {
+            // save the file
+            save();
+        } catch (Exception e) {
+            throw new AuthorityAccessException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public void ungroupUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
+        // get the user
+        final User user = getUser(dn);
+
+        // ensure the user was located
+        if (user == null) {
+            throw new UnknownIdentityException(String.format("User DN not found: %s.", dn));
+        }
+
+        // remove the users group
+        user.setGroup(null);
+
+        try {
+            // save the file
+            save();
+        } catch (Exception e) {
+            throw new AuthorityAccessException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public void ungroup(String group) throws AuthorityAccessException {
+        // get the user group
+        final Collection<User> userGroup = getUserGroup(group);
+
+        // ensure the user group was located
+        if (userGroup == null) {
+            return;
+        }
+
+        // update each user group
+        for (final User user : userGroup) {
+            user.setGroup(null);
+        }
+
+        try {
+            // save the file
+            save();
+        } catch (Exception e) {
+            throw new AuthorityAccessException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public String getGroupForUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
+        // get the user
+        final User user = getUser(dn);
+
+        // ensure the user was located
+        if (user == null) {
+            throw new UnknownIdentityException(String.format("User DN not found: %s.", dn));
+        }
+
+        return user.getGroup();
+    }
+
+    @Override
+    public void revokeGroup(String group) throws UnknownIdentityException, AuthorityAccessException {
+        // get the user group
+        final Collection<User> userGroup = getUserGroup(group);
+
+        // ensure the user group was located
+        if (userGroup == null) {
+            throw new UnknownIdentityException(String.format("User group not found: %s.", group));
+        }
+
+        // remove each user in the group
+        for (final User user : userGroup) {
+            users.getUser().remove(user);
+        }
+
+        try {
+            // save the file
+            save();
+        } catch (Exception e) {
+            throw new AuthorityAccessException(e.getMessage(), e);
+        }
+    }
+
+    /**
+     * Grants access to download content regardless of FlowFile attributes.
+     */
+    @Override
+    public DownloadAuthorization authorizeDownload(List<String> dnChain, Map<String, String> attributes) throws UnknownIdentityException, AuthorityAccessException {
+        return DownloadAuthorization.approved();
+    }
+
+    private User getUser(String dn) throws UnknownIdentityException {
+        // ensure the DN was specified
+        if (dn == null) {
+            throw new UnknownIdentityException("User DN not specified.");
+        }
+
+        // attempt to get the user and ensure it was located
+        User desiredUser = null;
+        for (final User user : users.getUser()) {
+            if (dn.equalsIgnoreCase(user.getDn())) {
+                desiredUser = user;
+                break;
+            }
+        }
+
+        return desiredUser;
+    }
+
+    private Collection<User> getUserGroup(String group) throws UnknownIdentityException {
+        // ensure the DN was specified
+        if (group == null) {
+            throw new UnknownIdentityException("User group not specified.");
+        }
+
+        // get all users with this group
+        Collection<User> userGroup = null;
+        for (final User user : users.getUser()) {
+            if (group.equals(user.getGroup())) {
+                if (userGroup == null) {
+                    userGroup = new HashSet<>();
+                }
+                userGroup.add(user);
+            }
+        }
+
+        return userGroup;
+    }
+
+    private void save() throws Exception {
+        final Marshaller marshaller = JAXB_CONTEXT.createMarshaller();
+        marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, Boolean.TRUE);
+
+        // save users to restore directory before primary directory
+        if (restoreUsersFile != null) {
+            marshaller.marshal(users, restoreUsersFile);
+        }
+
+        // save users to primary directory
+        marshaller.marshal(users, usersFile);
+    }
+
+    @AuthorityProviderContext
+    public void setNiFiProperties(NiFiProperties properties) {
+        this.properties = properties;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/3f4ac315/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/src/main/resources/META-INF/services/org.apache.nifi.authorization.AuthorityProvider
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/src/main/resources/META-INF/services/org.apache.nifi.authorization.AuthorityProvider b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/src/main/resources/META-INF/services/org.apache.nifi.authorization.AuthorityProvider
new file mode 100755
index 0000000..93d2941
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/src/main/resources/META-INF/services/org.apache.nifi.authorization.AuthorityProvider
@@ -0,0 +1,15 @@
+# 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.
+org.apache.nifi.authorization.FileAuthorizationProvider

http://git-wip-us.apache.org/repos/asf/nifi/blob/3f4ac315/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/src/main/xsd/users.xsd
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/src/main/xsd/users.xsd b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/src/main/xsd/users.xsd
new file mode 100644
index 0000000..4ee1e17
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/src/main/xsd/users.xsd
@@ -0,0 +1,64 @@
+<?xml version="1.0"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+      http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema">
+    <!-- role -->
+    <xs:complexType name="Role">
+        <xs:attribute name="name">
+            <xs:simpleType>
+                <xs:restriction base="xs:string">
+                    <xs:enumeration value="ROLE_MONITOR"/>
+                    <xs:enumeration value="ROLE_PROVENANCE"/>
+                    <xs:enumeration value="ROLE_DFM"/>
+                    <xs:enumeration value="ROLE_ADMIN"/>
+                    <xs:enumeration value="ROLE_PROXY"/>
+                    <xs:enumeration value="ROLE_NIFI"/>
+                </xs:restriction>
+            </xs:simpleType>
+        </xs:attribute>
+    </xs:complexType>
+
+    <!-- user -->
+    <xs:complexType name="User">
+        <xs:sequence>
+            <xs:element name="role" type="Role" minOccurs="0" maxOccurs="unbounded"/>
+        </xs:sequence>
+        <xs:attribute name="dn">
+            <xs:simpleType>
+                <xs:restriction base="xs:string">
+                    <xs:minLength value="1"/>
+                    <xs:pattern value=".*[^\s].*"/>
+                </xs:restriction>
+            </xs:simpleType>
+        </xs:attribute>
+        <xs:attribute name="group">
+            <xs:simpleType>
+                <xs:restriction base="xs:string">
+                    <xs:minLength value="1"/>
+                    <xs:pattern value=".*[^\s].*"/>
+                </xs:restriction>
+            </xs:simpleType>
+        </xs:attribute>
+    </xs:complexType>
+
+    <!-- users -->
+    <xs:element name="users">
+        <xs:complexType>
+            <xs:sequence>
+                <xs:element name="user" type="User" minOccurs="0" maxOccurs="unbounded"/>
+            </xs:sequence>
+        </xs:complexType>
+    </xs:element>
+</xs:schema>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/3f4ac315/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/src/test/java/org/apache/nifi/authorization/FileAuthorizationProviderTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/src/test/java/org/apache/nifi/authorization/FileAuthorizationProviderTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/src/test/java/org/apache/nifi/authorization/FileAuthorizationProviderTest.java
new file mode 100644
index 0000000..7428500
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/src/test/java/org/apache/nifi/authorization/FileAuthorizationProviderTest.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.authorization;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import org.apache.nifi.authorization.exception.ProviderCreationException;
+import org.apache.nifi.util.file.FileUtils;
+import org.apache.nifi.util.NiFiProperties;
+import org.junit.After;
+import static org.junit.Assert.assertEquals;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.Ignore;
+import org.mockito.Mockito;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+@Ignore
+public class FileAuthorizationProviderTest {
+
+    private FileAuthorizationProvider provider;
+
+    private File primary;
+
+    private File restore;
+
+    private NiFiProperties mockProperties;
+
+    private AuthorityProviderConfigurationContext mockConfigurationContext;
+
+    @Before
+    public void setup() throws IOException {
+
+        primary = new File("target/primary/users.txt");
+        restore = new File("target/restore/users.txt");
+
+        System.out.println("absolute path: " + primary.getAbsolutePath());
+
+        mockProperties = mock(NiFiProperties.class);
+        when(mockProperties.getRestoreDirectory()).thenReturn(restore.getParentFile());
+
+        mockConfigurationContext = mock(AuthorityProviderConfigurationContext.class);
+        when(mockConfigurationContext.getProperty(Mockito.eq("Authorized Users File"))).thenReturn(primary.getPath());
+
+        provider = new FileAuthorizationProvider();
+        provider.setNiFiProperties(mockProperties);
+        provider.initialize(null);
+    }
+
+    @After
+    public void cleanup() throws Exception {
+        deleteFile(primary);
+        deleteFile(restore);
+    }
+
+    private boolean deleteFile(final File file) {
+        if (file.isDirectory()) {
+            FileUtils.deleteFilesInDir(file, null, null, true, true);
+        }
+        return FileUtils.deleteFile(file, null, 10);
+    }
+
+    @Test
+    public void testPostContructionWhenRestoreDoesNotExist() throws Exception {
+
+        byte[] primaryBytes = "<users/>".getBytes();
+        FileOutputStream fos = new FileOutputStream(primary);
+        fos.write(primaryBytes);
+        fos.close();
+
+        provider.onConfigured(mockConfigurationContext);
+        assertEquals(primary.length(), restore.length());
+    }
+
+    @Test
+    public void testPostContructionWhenPrimaryDoesNotExist() throws Exception {
+
+        byte[] restoreBytes = "<users/>".getBytes();
+        FileOutputStream fos = new FileOutputStream(restore);
+        fos.write(restoreBytes);
+        fos.close();
+
+        provider.onConfigured(mockConfigurationContext);
+        assertEquals(restore.length(), primary.length());
+
+    }
+
+    @Test(expected = ProviderCreationException.class)
+    public void testPostContructionWhenPrimaryDifferentThanRestore() throws Exception {
+
+        byte[] primaryBytes = "<users></users>".getBytes();
+        FileOutputStream fos = new FileOutputStream(primary);
+        fos.write(primaryBytes);
+        fos.close();
+
+        byte[] restoreBytes = "<users/>".getBytes();
+        fos = new FileOutputStream(restore);
+        fos.write(restoreBytes);
+        fos.close();
+
+        provider.onConfigured(mockConfigurationContext);
+    }
+
+    @Test
+    public void testPostContructionWhenPrimaryAndBackupDoNotExist() throws Exception {
+
+        provider.onConfigured(mockConfigurationContext);
+        assertEquals(0, restore.length());
+        assertEquals(restore.length(), primary.length());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/3f4ac315/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 f06012c..db0b35e 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,8 +16,15 @@
  */
 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.Authorizer;
+
+import org.apache.nifi.authorization.AuthorityProvider;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.repository.ContentRepository;
 import org.apache.nifi.controller.repository.FlowFileRepository;
@@ -27,16 +34,10 @@ 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).
  *
@@ -57,7 +58,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(Authorizer.class, new HashSet<Class>());
+        definitionMap.put(AuthorityProvider.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/3f4ac315/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 93f73eb..9e9bd03 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,8 +16,16 @@
  */
 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.Authorizer;
+
+import org.apache.nifi.authorization.AuthorityProvider;
 import org.apache.nifi.components.Validator;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.repository.ContentRepository;
@@ -32,14 +40,6 @@ 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(Authorizer.class);
+        narSpecificClasses.add(AuthorityProvider.class);
         narSpecificClasses.add(LoginIdentityProvider.class);
         narSpecificClasses.add(ProvenanceEventRepository.class);
         narSpecificClasses.add(ComponentStatusRepository.class);

http://git-wip-us.apache.org/repos/asf/nifi/blob/3f4ac315/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
new file mode 100644
index 0000000..cb68e15
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/authority-providers.xml
@@ -0,0 +1,43 @@
+<?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/3f4ac315/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
index 054ddec..6b07165 100644
--- 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
@@ -17,7 +17,7 @@
     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 authorizers.xml for configuration details.
+    file and authority-providers.xml for configuration details.
     
     Available roles:
         ROLE_MONITOR        - for users - read only access to flow

http://git-wip-us.apache.org/repos/asf/nifi/blob/3f4ac315/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
deleted file mode 100644
index 5d7db16..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/authorizers.xml
+++ /dev/null
@@ -1,28 +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.
--->
-<authorizers>
-    <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>
-</authorizers>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/3f4ac315/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 3724c1c..beb71c1 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.authorizer.configuration.file=${nifi.authorizer.configuration.file}
+nifi.authority.provider.configuration.file=${nifi.authority.provider.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}
@@ -138,7 +138,7 @@ 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.authorizer=${nifi.security.user.authorizer}
+nifi.security.user.authority.provider=${nifi.security.user.authority.provider}
 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

http://git-wip-us.apache.org/repos/asf/nifi/blob/3f4ac315/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 589cf7f..66fd303 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,7 +16,30 @@
  */
 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.components.ValidationResult;
 import org.apache.nifi.connectable.ConnectableType;
 import org.apache.nifi.controller.AbstractPort;
@@ -41,27 +64,10 @@ 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";
@@ -349,8 +355,67 @@ public class StandardRootGroupPort extends AbstractPort implements RootGroupPort
             return new StandardPortAuthorizationResult(false, "User DN is not known");
         }
 
-        // TODO - Replace with call to Authorizer to authorize site to site data transfer
-        return new StandardPortAuthorizationResult(true, "User is Authorized");
+        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);
+        }
     }
 
     public static class StandardPortAuthorizationResult implements PortAuthorizationResult {

http://git-wip-us.apache.org/repos/asf/nifi/blob/3f4ac315/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 dee219e..cfe18c5 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
@@ -50,6 +50,8 @@ import org.apache.nifi.web.api.dto.RevisionDTO;
 import org.apache.nifi.web.api.dto.SnippetDTO;
 import org.apache.nifi.web.api.dto.SystemDiagnosticsDTO;
 import org.apache.nifi.web.api.dto.TemplateDTO;
+import org.apache.nifi.web.api.dto.UserDTO;
+import org.apache.nifi.web.api.dto.UserGroupDTO;
 import org.apache.nifi.web.api.dto.action.ActionDTO;
 import org.apache.nifi.web.api.dto.action.HistoryDTO;
 import org.apache.nifi.web.api.dto.action.HistoryQueryDTO;
@@ -66,6 +68,7 @@ import org.apache.nifi.web.api.dto.status.ProcessorStatusDTO;
 import org.apache.nifi.web.api.dto.status.RemoteProcessGroupStatusDTO;
 import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
 
+import java.util.Collection;
 import java.util.Date;
 import java.util.Set;
 
@@ -1426,6 +1429,84 @@ public interface NiFiServiceFacade {
     ConfigurationSnapshot<Void> deleteSnippet(Revision revision, String snippetId);
 
     // ----------------------------------------
+    // User methods
+    // ----------------------------------------
+    /**
+     * Gets the user with the specified id.
+     *
+     * @param userId The user id
+     * @return user
+     */
+    UserDTO getUser(String userId);
+
+    /**
+     * Gets all of the users registered with this controller.
+     *
+     * @param grouped grouped
+     * @return user
+     */
+    Collection<UserDTO> getUsers(Boolean grouped);
+
+    /**
+     * Creates a new account request.
+     *
+     * @return user
+     */
+    UserDTO createUser();
+
+    /**
+     * Updates the specified user accordingly.
+     *
+     * @param user The user to update
+     * @return user
+     */
+    UserDTO updateUser(UserDTO user);
+
+    /**
+     * Invalidates the specified user.
+     *
+     * @param userId user
+     */
+    void invalidateUser(String userId);
+
+    /**
+     * Invalidates the specified user accounts and all accounts associated with this group.
+     *
+     * @param userGroup group
+     * @param userIds id
+     */
+    void invalidateUserGroup(String userGroup, Set<String> userIds);
+
+    /**
+     * Deletes the specified user.
+     *
+     * @param userId user id
+     */
+    void deleteUser(String userId);
+
+    /**
+     * Updates a user group with the specified group and comprised of the specified users.
+     *
+     * @param userGroup group
+     * @return group
+     */
+    UserGroupDTO updateUserGroup(UserGroupDTO userGroup);
+
+    /**
+     * Ungroups the specified user.
+     *
+     * @param userId id
+     */
+    void removeUserFromGroup(String userId);
+
+    /**
+     * Deletes the specified user group.
+     *
+     * @param userGroup group
+     */
+    void removeUserGroup(String userGroup);
+
+    // ----------------------------------------
     // Cluster methods
     // ----------------------------------------
     /**

http://git-wip-us.apache.org/repos/asf/nifi/blob/3f4ac315/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiWebApiSecurityConfiguration.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiWebApiSecurityConfiguration.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiWebApiSecurityConfiguration.java
index c85835b..fd44636 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiWebApiSecurityConfiguration.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiWebApiSecurityConfiguration.java
@@ -18,14 +18,15 @@ package org.apache.nifi.web;
 
 import org.apache.nifi.admin.service.UserService;
 import org.apache.nifi.util.NiFiProperties;
+import org.apache.nifi.web.security.NiFiAuthenticationProvider;
 import org.apache.nifi.web.security.anonymous.NiFiAnonymousUserFilter;
 import org.apache.nifi.web.security.jwt.JwtAuthenticationFilter;
-import org.apache.nifi.web.security.jwt.JwtAuthenticationProvider;
+import org.apache.nifi.web.security.jwt.JwtService;
 import org.apache.nifi.web.security.node.NodeAuthorizedUserFilter;
 import org.apache.nifi.web.security.otp.OtpAuthenticationFilter;
-import org.apache.nifi.web.security.otp.OtpAuthenticationProvider;
+import org.apache.nifi.web.security.otp.OtpService;
+import org.apache.nifi.web.security.token.NiFiAuthorizationRequestToken;
 import org.apache.nifi.web.security.x509.X509AuthenticationFilter;
-import org.apache.nifi.web.security.x509.X509AuthenticationProvider;
 import org.apache.nifi.web.security.x509.X509CertificateExtractor;
 import org.apache.nifi.web.security.x509.X509IdentityProvider;
 import org.slf4j.Logger;
@@ -41,6 +42,7 @@ import org.springframework.security.config.annotation.web.builders.WebSecurity;
 import org.springframework.security.config.annotation.web.configuration.EnableWebSecurity;
 import org.springframework.security.config.annotation.web.configuration.WebSecurityConfigurerAdapter;
 import org.springframework.security.config.http.SessionCreationPolicy;
+import org.springframework.security.core.userdetails.AuthenticationUserDetailsService;
 import org.springframework.security.web.authentication.AnonymousAuthenticationFilter;
 
 /**
@@ -54,20 +56,16 @@ public class NiFiWebApiSecurityConfiguration extends WebSecurityConfigurerAdapte
 
     private NiFiProperties properties;
     private UserService userService;
-
-    private NodeAuthorizedUserFilter nodeAuthorizedUserFilter;
-
-    private X509AuthenticationFilter x509AuthenticationFilter;
+    private AuthenticationUserDetailsService authenticationUserDetailsService;
+    private JwtService jwtService;
+    private OtpService otpService;
     private X509CertificateExtractor certificateExtractor;
     private X509IdentityProvider certificateIdentityProvider;
-    private X509AuthenticationProvider x509AuthenticationProvider;
 
+    private NodeAuthorizedUserFilter nodeAuthorizedUserFilter;
     private JwtAuthenticationFilter jwtAuthenticationFilter;
-    private JwtAuthenticationProvider jwtAuthenticationProvider;
-
     private OtpAuthenticationFilter otpAuthenticationFilter;
-    private OtpAuthenticationProvider otpAuthenticationProvider;
-
+    private X509AuthenticationFilter x509AuthenticationFilter;
     private NiFiAnonymousUserFilter anonymousAuthenticationFilter;
 
     public NiFiWebApiSecurityConfiguration() {
@@ -97,17 +95,17 @@ public class NiFiWebApiSecurityConfiguration extends WebSecurityConfigurerAdapte
         // cluster authorized user
         http.addFilterBefore(nodeAuthorizedUserFilterBean(), AnonymousAuthenticationFilter.class);
 
+        // anonymous
+        http.anonymous().authenticationFilter(anonymousFilterBean());
+
         // x509
-        http.addFilterBefore(x509FilterBean(), AnonymousAuthenticationFilter.class);
+        http.addFilterAfter(x509FilterBean(), AnonymousAuthenticationFilter.class);
 
         // jwt
-        http.addFilterBefore(jwtFilterBean(), AnonymousAuthenticationFilter.class);
+        http.addFilterAfter(jwtFilterBean(), AnonymousAuthenticationFilter.class);
 
         // otp
-        http.addFilterBefore(otpFilterBean(), AnonymousAuthenticationFilter.class);
-
-        // anonymous
-        http.anonymous().authenticationFilter(anonymousFilterBean());
+        http.addFilterAfter(otpFilterBean(), AnonymousAuthenticationFilter.class);
     }
 
     @Bean
@@ -119,10 +117,7 @@ public class NiFiWebApiSecurityConfiguration extends WebSecurityConfigurerAdapte
 
     @Override
     protected void configure(AuthenticationManagerBuilder auth) throws Exception {
-        auth
-                .authenticationProvider(x509AuthenticationProvider)
-                .authenticationProvider(jwtAuthenticationProvider)
-                .authenticationProvider(otpAuthenticationProvider);
+        auth.authenticationProvider(new NiFiAuthenticationProvider(authenticationUserDetailsService));
     }
 
     @Bean
@@ -142,6 +137,7 @@ public class NiFiWebApiSecurityConfiguration extends WebSecurityConfigurerAdapte
             jwtAuthenticationFilter = new JwtAuthenticationFilter();
             jwtAuthenticationFilter.setProperties(properties);
             jwtAuthenticationFilter.setAuthenticationManager(authenticationManager());
+            jwtAuthenticationFilter.setJwtService(jwtService);
         }
         return jwtAuthenticationFilter;
     }
@@ -152,6 +148,7 @@ public class NiFiWebApiSecurityConfiguration extends WebSecurityConfigurerAdapte
             otpAuthenticationFilter = new OtpAuthenticationFilter();
             otpAuthenticationFilter.setProperties(properties);
             otpAuthenticationFilter.setAuthenticationManager(authenticationManager());
+            otpAuthenticationFilter.setOtpService(otpService);
         }
         return otpAuthenticationFilter;
     }
@@ -162,6 +159,7 @@ public class NiFiWebApiSecurityConfiguration extends WebSecurityConfigurerAdapte
             x509AuthenticationFilter = new X509AuthenticationFilter();
             x509AuthenticationFilter.setProperties(properties);
             x509AuthenticationFilter.setCertificateExtractor(certificateExtractor);
+            x509AuthenticationFilter.setCertificateIdentityProvider(certificateIdentityProvider);
             x509AuthenticationFilter.setAuthenticationManager(authenticationManager());
         }
         return x509AuthenticationFilter;
@@ -177,6 +175,11 @@ public class NiFiWebApiSecurityConfiguration extends WebSecurityConfigurerAdapte
     }
 
     @Autowired
+    public void setUserDetailsService(AuthenticationUserDetailsService<NiFiAuthorizationRequestToken> userDetailsService) {
+        this.authenticationUserDetailsService = userDetailsService;
+    }
+
+    @Autowired
     public void setUserService(UserService userService) {
         this.userService = userService;
     }
@@ -187,18 +190,13 @@ public class NiFiWebApiSecurityConfiguration extends WebSecurityConfigurerAdapte
     }
 
     @Autowired
-    public void setJwtAuthenticationProvider(JwtAuthenticationProvider jwtAuthenticationProvider) {
-        this.jwtAuthenticationProvider = jwtAuthenticationProvider;
-    }
-
-    @Autowired
-    public void setOtpAuthenticationProvider(OtpAuthenticationProvider otpAuthenticationProvider) {
-        this.otpAuthenticationProvider = otpAuthenticationProvider;
+    public void setJwtService(JwtService jwtService) {
+        this.jwtService = jwtService;
     }
 
     @Autowired
-    public void setX509AuthenticationProvider(X509AuthenticationProvider x509AuthenticationProvider) {
-        this.x509AuthenticationProvider = x509AuthenticationProvider;
+    public void setOtpService(OtpService otpService) {
+        this.otpService = otpService;
     }
 
     @Autowired

http://git-wip-us.apache.org/repos/asf/nifi/blob/3f4ac315/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiContentAccess.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiContentAccess.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiContentAccess.java
index a9559a3..afaf3ed 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiContentAccess.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiContentAccess.java
@@ -20,6 +20,7 @@ import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse.Status;
 import com.sun.jersey.core.util.MultivaluedMapImpl;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.authorization.Authority;
 import org.apache.nifi.cluster.manager.NodeResponse;
 import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
 import org.apache.nifi.cluster.manager.impl.WebClusterManager;
@@ -28,6 +29,7 @@ import org.apache.nifi.cluster.protocol.NodeIdentifier;
 import org.apache.nifi.controller.repository.claim.ContentDirection;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.security.user.NiFiUserDetails;
+import org.apache.nifi.web.security.user.NiFiUserUtils;
 import org.apache.nifi.web.util.WebUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -178,19 +180,19 @@ public class StandardNiFiContentAccess implements ContentAccess {
     }
 
     private DownloadableContent getFlowFileContent(final String groupId, final String connectionId, final String flowfileId, final String dataUri) {
-        // TODO - ensure the user is authorized - not checking with @PreAuthorized annotation as aspect not trigger on call within a class
-//        if (!NiFiUserUtils.getAuthorities().contains(Authority.ROLE_DFM.toString())) {
-//            throw new AccessDeniedException("Access is denied.");
-//        }
+        // ensure the user is authorized as DFM - not checking with @PreAuthorized annotation as aspect not trigger on call within a class
+        if (!NiFiUserUtils.getAuthorities().contains(Authority.ROLE_DFM.toString())) {
+            throw new AccessDeniedException("Access is denied.");
+        }
 
         return serviceFacade.getContent(groupId, connectionId, flowfileId, dataUri);
     }
 
     private DownloadableContent getProvenanceEventContent(final Long eventId, final String dataUri, final ContentDirection direction) {
-        // TODO - ensure the user is authorized - not checking with @PreAuthorized annotation as aspect not trigger on call within a class
-//        if (!NiFiUserUtils.getAuthorities().contains(Authority.ROLE_PROVENANCE.toString())) {
-//            throw new AccessDeniedException("Access is denied.");
-//        }
+        // ensure the user is authorized as Provenance - not checking with @PreAuthorized annotation as aspect not trigger on call within a class
+        if (!NiFiUserUtils.getAuthorities().contains(Authority.ROLE_PROVENANCE.toString())) {
+            throw new AccessDeniedException("Access is denied.");
+        }
 
         return serviceFacade.getContent(eventId, dataUri, direction);
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/3f4ac315/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 7fe2f06..4fdda06 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
@@ -16,14 +16,17 @@
  */
 package org.apache.nifi.web;
 
+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;
 import org.apache.nifi.action.Operation;
 import org.apache.nifi.action.details.FlowChangePurgeDetails;
+import org.apache.nifi.admin.service.AccountNotFoundException;
 import org.apache.nifi.admin.service.AuditService;
 import org.apache.nifi.admin.service.UserService;
+import org.apache.nifi.authorization.Authority;
 import org.apache.nifi.cluster.context.ClusterContext;
 import org.apache.nifi.cluster.context.ClusterContextThreadLocal;
 import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
@@ -61,7 +64,9 @@ import org.apache.nifi.remote.RootGroupPort;
 import org.apache.nifi.reporting.Bulletin;
 import org.apache.nifi.reporting.BulletinQuery;
 import org.apache.nifi.reporting.BulletinRepository;
+import org.apache.nifi.user.AccountStatus;
 import org.apache.nifi.user.NiFiUser;
+import org.apache.nifi.user.NiFiUserGroup;
 import org.apache.nifi.util.FormatUtils;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.api.dto.BulletinBoardDTO;
@@ -101,6 +106,8 @@ import org.apache.nifi.web.api.dto.RevisionDTO;
 import org.apache.nifi.web.api.dto.SnippetDTO;
 import org.apache.nifi.web.api.dto.SystemDiagnosticsDTO;
 import org.apache.nifi.web.api.dto.TemplateDTO;
+import org.apache.nifi.web.api.dto.UserDTO;
+import org.apache.nifi.web.api.dto.UserGroupDTO;
 import org.apache.nifi.web.api.dto.action.ActionDTO;
 import org.apache.nifi.web.api.dto.action.HistoryDTO;
 import org.apache.nifi.web.api.dto.action.HistoryQueryDTO;
@@ -128,10 +135,12 @@ import org.apache.nifi.web.dao.RemoteProcessGroupDAO;
 import org.apache.nifi.web.dao.ReportingTaskDAO;
 import org.apache.nifi.web.dao.SnippetDAO;
 import org.apache.nifi.web.dao.TemplateDAO;
+import org.apache.nifi.web.security.user.NewAccountRequest;
 import org.apache.nifi.web.security.user.NiFiUserUtils;
 import org.apache.nifi.web.util.SnippetUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.springframework.security.access.AccessDeniedException;
 
 import javax.ws.rs.WebApplicationException;
 import java.nio.charset.StandardCharsets;
@@ -139,6 +148,8 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
@@ -1887,6 +1898,120 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
     }
 
     @Override
+    public void invalidateUser(String userId) {
+        try {
+            userService.invalidateUserAccount(userId);
+        } catch (final AccountNotFoundException anfe) {
+            // ignore
+        }
+    }
+
+    @Override
+    public void invalidateUserGroup(String userGroup, Set<String> userIds) {
+        // invalidates any user currently associated with this group
+        if (userGroup != null) {
+            userService.invalidateUserGroupAccount(userGroup);
+        }
+
+        // invalidates any user that will be associated with this group
+        if (userIds != null) {
+            for (final String userId : userIds) {
+                invalidateUser(userId);
+            }
+        }
+    }
+
+    @Override
+    public UserDTO createUser() {
+        NewAccountRequest newAccountRequest = NiFiUserUtils.getNewAccountRequest();
+
+        // log the new user account request
+        logger.info("Requesting new user account for " + newAccountRequest.getUsername());
+
+        // get the justification
+        String justification = newAccountRequest.getJustification();
+        if (justification == null) {
+            justification = StringUtils.EMPTY;
+        }
+
+        // create the pending user account
+        return dtoFactory.createUserDTO(userService.createPendingUserAccount(newAccountRequest.getUsername(), justification));
+    }
+
+    @Override
+    public UserDTO updateUser(UserDTO userDto) {
+        NiFiUser user;
+
+        // attempt to parse the user id
+        final String id = userDto.getId();
+
+        // determine the authorities that have been specified in the request
+        Set<Authority> authorities = null;
+        if (userDto.getAuthorities() != null) {
+            authorities = Authority.convertRawAuthorities(userDto.getAuthorities());
+        }
+
+        // if the account status isn't specified or isn't changing
+        final AccountStatus accountStatus = AccountStatus.valueOfStatus(userDto.getStatus());
+        if (accountStatus == null || AccountStatus.ACTIVE.equals(accountStatus)) {
+            // ensure that authorities have been specified (may be empty, but not null)
+            if (authorities == null) {
+                throw new IllegalArgumentException("Authorities must be specified when updating an account.");
+            }
+
+            // update the user account
+            user = userService.update(id, authorities);
+        } else if (AccountStatus.DISABLED.equals(accountStatus)) {
+            // disable the account
+            user = userService.disable(id);
+        } else {
+            throw new IllegalArgumentException("Accounts cannot be marked pending.");
+        }
+
+        return dtoFactory.createUserDTO(user);
+    }
+
+    @Override
+    public void deleteUser(String userId) {
+        userService.deleteUser(userId);
+    }
+
+    @Override
+    public UserGroupDTO updateUserGroup(final UserGroupDTO userGroupDTO) {
+        NiFiUserGroup userGroup;
+
+        // convert the authorities
+        Set<Authority> authorities = null;
+        if (userGroupDTO.getAuthorities() != null) {
+            authorities = Authority.convertRawAuthorities(userGroupDTO.getAuthorities());
+        }
+
+        final AccountStatus accountStatus = AccountStatus.valueOfStatus(userGroupDTO.getStatus());
+        if (accountStatus == null || AccountStatus.ACTIVE.equals(accountStatus)) {
+            // update the user group
+            userGroup = userService.updateGroup(userGroupDTO.getGroup(), userGroupDTO.getUserIds(), authorities);
+        } else if (AccountStatus.DISABLED.equals(accountStatus)) {
+            // disable the accounts
+            userGroup = userService.disableGroup(userGroupDTO.getGroup());
+        } else {
+            throw new IllegalArgumentException("Accounts cannot be marked pending.");
+        }
+
+        // generate the user group dto
+        return dtoFactory.createUserGroupDTO(userGroup);
+    }
+
+    @Override
+    public void removeUserFromGroup(String userId) {
+        userService.ungroupUser(userId);
+    }
+
+    @Override
+    public void removeUserGroup(String userGroup) {
+        userService.ungroup(userGroup);
+    }
+
+    @Override
     public ProvenanceDTO submitProvenance(ProvenanceDTO query) {
         return controllerFacade.submitProvenance(query);
     }
@@ -1961,7 +2086,15 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
 
     @Override
     public ControllerStatusDTO getControllerStatus() {
-        return controllerFacade.getControllerStatus();
+        // get the controller status
+        final ControllerStatusDTO controllerStatus = controllerFacade.getControllerStatus();
+
+        // determine if there are any pending user accounts - only include if appropriate
+        if (NiFiUserUtils.getAuthorities().contains(Authority.ROLE_ADMIN.toString())) {
+            controllerStatus.setHasPendingAccounts(userService.hasPendingUserAccount());
+        }
+
+        return controllerStatus;
     }
 
     @Override
@@ -2196,8 +2329,18 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
             return true;
         }
 
-        // TODO - defer to authorizer to see if user is able to retrieve site-to-site details for the specified port
-        return true;
+        final Set<String> allowedUsers = port.getUserAccessControl();
+        if (allowedUsers.contains(user.getIdentity())) {
+            return true;
+        }
+
+        final String userGroup = user.getUserGroup();
+        if (userGroup == null) {
+            return false;
+        }
+
+        final Set<String> allowedGroups = port.getGroupAccessControl();
+        return allowedGroups.contains(userGroup);
     }
 
     @Override
@@ -2207,9 +2350,12 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
             throw new WebApplicationException(new Throwable("Unable to access details for current user."));
         }
 
-        // TODO - defer to authorizer to see if user is able to retrieve site-to-site details
-
-        // TODO - filter response for access to specific ports
+        // at this point we know that the user must have ROLE_NIFI because it's required
+        // get to the endpoint that calls this method but we'll check again anyways
+        final Set<Authority> authorities = user.getAuthorities();
+        if (!authorities.contains(Authority.ROLE_NIFI)) {
+            throw new AccessDeniedException("User must have the NiFi role in order to access these details.");
+        }
 
         // serialize the input ports this NiFi has access to
         final Set<PortDTO> inputPorts = new LinkedHashSet<>();
@@ -2547,6 +2693,82 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
     }
 
     @Override
+    public UserDTO getUser(String userId) {
+        // get the user
+        NiFiUser user = userService.getUserById(userId);
+
+        // ensure the user was found
+        if (user == null) {
+            throw new ResourceNotFoundException(String.format("Unable to find user with id '%s'.", userId));
+        }
+
+        return dtoFactory.createUserDTO(user);
+    }
+
+    @Override
+    public Collection<UserDTO> getUsers(Boolean grouped) {
+        // get the users
+        final Collection<NiFiUser> users = userService.getUsers();
+        final Collection<UserDTO> userDTOs = new HashSet<>();
+
+        if (grouped) {
+            final Map<String, UserDTO> groupedUserDTOs = new HashMap<>();
+
+            // group the users
+            for (final NiFiUser user : users) {
+                if (StringUtils.isNotBlank(user.getUserGroup())) {
+                    if (groupedUserDTOs.containsKey(user.getUserGroup())) {
+                        final UserDTO groupedUser = groupedUserDTOs.get(user.getUserGroup());
+                        groupedUser.setId(groupedUser.getId() + "," + String.valueOf(user.getId()));
+                        groupedUser.setUserName(groupedUser.getUserName() + ", " + user.getUserName());
+                        groupedUser.setDn(groupedUser.getDn() + ", " + user.getIdentity());
+                        groupedUser.setCreation(getOldestDate(groupedUser.getCreation(), user.getCreation()));
+                        groupedUser.setLastAccessed(getNewestDate(groupedUser.getLastAccessed(), user.getLastAccessed()));
+                        groupedUser.setLastVerified(getNewestDate(groupedUser.getLastVerified(), user.getLastVerified()));
+
+                        // only retain the justification if al users have the same justification
+                        if (groupedUser.getJustification() != null) {
+                            if (!groupedUser.getStatus().equals(user.getJustification())) {
+                                groupedUser.setJustification(null);
+                            }
+                        }
+
+                        // only retain the status if all users have the same status
+                        if (groupedUser.getStatus() != null) {
+                            if (!groupedUser.getStatus().equals(user.getStatus().toString())) {
+                                groupedUser.setStatus(null);
+                            }
+                        }
+
+                        // only retain the authorities if all users have the same authorities
+                        if (groupedUser.getAuthorities() != null) {
+                            final Set<String> groupAuthorities = new HashSet<>(groupedUser.getAuthorities());
+                            final Set<String> userAuthorities = Authority.convertAuthorities(user.getAuthorities());
+                            if (!CollectionUtils.isEqualCollection(groupAuthorities, userAuthorities)) {
+                                groupedUser.setAuthorities(null);
+                            }
+                        }
+                    } else {
+                        groupedUserDTOs.put(user.getUserGroup(), dtoFactory.createUserDTO(user));
+                    }
+                } else {
+                    userDTOs.add(dtoFactory.createUserDTO(user));
+                }
+            }
+
+            // add the grouped users
+            userDTOs.addAll(groupedUserDTOs.values());
+        } else {
+            // convert each into a DTOs
+            for (final NiFiUser user : users) {
+                userDTOs.add(dtoFactory.createUserDTO(user));
+            }
+        }
+
+        return userDTOs;
+    }
+
+    @Override
     public boolean isClustered() {
         return controllerFacade.isClustered();
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/3f4ac315/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java
index 3a57d1d..cae1175 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java
@@ -160,7 +160,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
     @Override
     @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
     public String getCurrentUserDn() {
-        String userIdentity = NiFiUser.ANONYMOUS.getIdentity();
+        String userIdentity = NiFiUser.ANONYMOUS_USER_IDENTITY;
 
         final NiFiUser user = NiFiUserUtils.getNiFiUser();
         if (user != null) {
@@ -173,7 +173,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
     @Override
     @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
     public String getCurrentUserName() {
-        String userName = NiFiUser.ANONYMOUS.getIdentity();
+        String userName = NiFiUser.ANONYMOUS_USER_IDENTITY;
 
         final NiFiUser user = NiFiUserUtils.getNiFiUser();
         if (user != null) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/3f4ac315/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java
index cd8d0c7..9667ad6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java
@@ -131,7 +131,7 @@ public class StandardNiFiWebContext implements NiFiWebContext {
     @Override
     @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
     public String getCurrentUserDn() {
-        String userIdentity = NiFiUser.ANONYMOUS.getIdentity();
+        String userIdentity = NiFiUser.ANONYMOUS_USER_IDENTITY;
 
         final NiFiUser user = NiFiUserUtils.getNiFiUser();
         if (user != null) {
@@ -144,7 +144,7 @@ public class StandardNiFiWebContext implements NiFiWebContext {
     @Override
     @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
     public String getCurrentUserName() {
-        String userName = NiFiUser.ANONYMOUS.getIdentity();
+        String userName = NiFiUser.ANONYMOUS_USER_IDENTITY;
 
         final NiFiUser user = NiFiUserUtils.getNiFiUser();
         if (user != null) {