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

[17/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-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/ClusterManagerAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/ClusterManagerAuthorizationProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/ClusterManagerAuthorizationProvider.java
deleted file mode 100644
index 2b3b38c..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/ClusterManagerAuthorizationProvider.java
+++ /dev/null
@@ -1,225 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.cluster.authorization;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.authorization.AuthorityProviderConfigurationContext;
-import org.apache.nifi.authorization.AuthorityProviderInitializationContext;
-import org.apache.nifi.authorization.FileAuthorizationProvider;
-import org.apache.nifi.authorization.annotation.AuthorityProviderContext;
-import org.apache.nifi.authorization.exception.ProviderCreationException;
-import org.apache.nifi.authorization.exception.ProviderDestructionException;
-import org.apache.nifi.cluster.authorization.protocol.message.DoesDnExistMessage;
-import org.apache.nifi.cluster.authorization.protocol.message.GetAuthoritiesMessage;
-import org.apache.nifi.cluster.authorization.protocol.message.GetGroupForUserMessage;
-import org.apache.nifi.cluster.authorization.protocol.message.ProtocolMessage;
-import static org.apache.nifi.cluster.authorization.protocol.message.ProtocolMessage.MessageType.DOES_DN_EXIST;
-import static org.apache.nifi.cluster.authorization.protocol.message.ProtocolMessage.MessageType.GET_AUTHORITIES;
-import static org.apache.nifi.cluster.authorization.protocol.message.ProtocolMessage.MessageType.GET_GROUP_FOR_USER;
-import org.apache.nifi.cluster.authorization.protocol.message.jaxb.JaxbProtocolUtils;
-import org.apache.nifi.cluster.manager.impl.WebClusterManager;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
-import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller;
-import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
-import org.apache.nifi.io.socket.ServerSocketConfiguration;
-import org.apache.nifi.io.socket.SocketListener;
-import org.apache.nifi.io.socket.SocketUtils;
-import org.apache.nifi.io.socket.multicast.DiscoverableService;
-import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
-import org.apache.nifi.logging.NiFiLog;
-import org.apache.nifi.util.NiFiProperties;
-import static org.apache.nifi.util.NiFiProperties.CLUSTER_MANAGER_ADDRESS;
-import org.apache.nifi.util.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.springframework.beans.BeansException;
-import org.springframework.context.ApplicationContext;
-import org.springframework.context.ApplicationContextAware;
-
-/**
- * Provides authorities for the NCM in clustered environments. Communication
- * occurs over TCP/IP sockets. All method calls are deferred to the
- * FileAuthorizationProvider.
- */
-public class ClusterManagerAuthorizationProvider extends FileAuthorizationProvider implements AuthorityProvider, ApplicationContextAware {
-
-    public static final String AUTHORITY_PROVIDER_SERVIVE_NAME = "cluster-authority-provider";
-
-    private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(ClusterManagerAuthorizationProvider.class));
-    private static final String CLUSTER_MANAGER_AUTHORITY_PROVIDER_PORT = "Authority Provider Port";
-    private static final String CLUSTER_MANAGER_AUTHORITY_PROVIDER_THREADS = "Authority Provider Threads";
-    private static final int DEFAULT_CLUSTER_MANAGER_AUTHORITY_PROVIDER_THREADS = 10;
-
-    private WebClusterManager clusterManager;
-    private ProtocolContext<ProtocolMessage> authorityProviderProtocolContext;
-    private SocketListener socketListener;
-    private NiFiProperties properties;
-    private ApplicationContext applicationContext;
-
-    @Override
-    public void initialize(final AuthorityProviderInitializationContext initializationContext) throws ProviderCreationException {
-        super.initialize(initializationContext);
-    }
-
-    @Override
-    public void onConfigured(final AuthorityProviderConfigurationContext configurationContext) throws ProviderCreationException {
-        super.onConfigured(configurationContext);
-
-        // get the socket address of the cluster authority provider
-        final InetSocketAddress clusterAuthorityProviderAddress = getClusterManagerAuthorityProviderAddress(configurationContext);
-
-        // get the cluster manager
-        clusterManager = applicationContext.getBean("clusterManager", WebClusterManager.class);
-
-        // if using multicast, then the authority provider's service is broadcasted
-        if (properties.getClusterProtocolUseMulticast()) {
-
-            // create the authority provider service for discovery
-            final DiscoverableService clusterAuthorityProviderService = new DiscoverableServiceImpl(AUTHORITY_PROVIDER_SERVIVE_NAME, clusterAuthorityProviderAddress);
-
-            // register the authority provider service with the cluster manager
-            clusterManager.addBroadcastedService(clusterAuthorityProviderService);
-        }
-
-        // get the number of protocol listening thread
-        final int numThreads = getClusterManagerAuthorityProviderThreads(configurationContext);
-
-        // the server socket configuration
-        final ServerSocketConfiguration configuration = applicationContext.getBean("protocolServerSocketConfiguration", ServerSocketConfiguration.class);
-
-        // the authority provider listens for node messages
-        socketListener = new SocketListener(numThreads, clusterAuthorityProviderAddress.getPort(), configuration) {
-            @Override
-            public void dispatchRequest(final Socket socket) {
-                ClusterManagerAuthorizationProvider.this.dispatchRequest(socket);
-            }
-        };
-
-        // start the socket listener
-        if (socketListener != null && !socketListener.isRunning()) {
-            try {
-                socketListener.start();
-            } catch (final IOException ioe) {
-                throw new ProviderCreationException("Failed to start Cluster Manager Authorization Provider due to: " + ioe, ioe);
-            }
-        }
-
-        // initialize the protocol context
-        authorityProviderProtocolContext = new JaxbProtocolContext<ProtocolMessage>(JaxbProtocolUtils.JAXB_CONTEXT);
-    }
-
-    @Override
-    public void preDestruction() throws ProviderDestructionException {
-        if (socketListener != null && socketListener.isRunning()) {
-            try {
-                socketListener.stop();
-            } catch (final IOException ioe) {
-                throw new ProviderDestructionException("Failed to stop Cluster Manager Authorization Provider due to: " + ioe, ioe);
-            }
-        }
-        super.preDestruction();
-    }
-
-    private int getClusterManagerAuthorityProviderThreads(final AuthorityProviderConfigurationContext configurationContext) {
-        try {
-            return Integer.parseInt(configurationContext.getProperty(CLUSTER_MANAGER_AUTHORITY_PROVIDER_THREADS));
-        } catch (NumberFormatException nfe) {
-            return DEFAULT_CLUSTER_MANAGER_AUTHORITY_PROVIDER_THREADS;
-        }
-    }
-
-    private InetSocketAddress getClusterManagerAuthorityProviderAddress(final AuthorityProviderConfigurationContext configurationContext) {
-        try {
-            String socketAddress = properties.getProperty(CLUSTER_MANAGER_ADDRESS);
-            if (StringUtils.isBlank(socketAddress)) {
-                socketAddress = "localhost";
-            }
-            return InetSocketAddress.createUnresolved(socketAddress, getClusterManagerAuthorityProviderPort(configurationContext));
-        } catch (Exception ex) {
-            throw new RuntimeException("Invalid manager authority provider address/port due to: " + ex, ex);
-        }
-    }
-
-    private Integer getClusterManagerAuthorityProviderPort(final AuthorityProviderConfigurationContext configurationContext) {
-        final String authorityProviderPort = configurationContext.getProperty(CLUSTER_MANAGER_AUTHORITY_PROVIDER_PORT);
-        if (authorityProviderPort == null || authorityProviderPort.trim().isEmpty()) {
-            throw new ProviderCreationException("The authority provider port must be specified.");
-        }
-
-        return Integer.parseInt(authorityProviderPort);
-    }
-
-    private void dispatchRequest(final Socket socket) {
-        try {
-            // unmarshall message
-            final ProtocolMessageUnmarshaller<ProtocolMessage> unmarshaller = authorityProviderProtocolContext.createUnmarshaller();
-            final ProtocolMessage request = unmarshaller.unmarshal(socket.getInputStream());
-            final ProtocolMessage response = request;
-
-            try {
-                switch (request.getType()) {
-                    case DOES_DN_EXIST: {
-                        final DoesDnExistMessage castedMsg = (DoesDnExistMessage) request;
-                        castedMsg.setResponse(doesDnExist(castedMsg.getDn()));
-                        break;
-                    }
-                    case GET_AUTHORITIES: {
-                        final GetAuthoritiesMessage castedMsg = (GetAuthoritiesMessage) request;
-                        castedMsg.setResponse(getAuthorities(castedMsg.getDn()));
-                        break;
-                    }
-                    case GET_GROUP_FOR_USER: {
-                        final GetGroupForUserMessage castedMsg = (GetGroupForUserMessage) request;
-                        castedMsg.setResponse(getGroupForUser(castedMsg.getDn()));
-                        break;
-                    }
-                    default: {
-                        throw new Exception("Unsupported Message Type: " + request.getType());
-                    }
-                }
-            } catch (final Exception ex) {
-                response.setExceptionClass(ex.getClass().getName());
-                response.setExceptionMessage(ex.getMessage());
-            }
-
-            final ProtocolMessageMarshaller<ProtocolMessage> marshaller = authorityProviderProtocolContext.createMarshaller();
-            marshaller.marshal(response, socket.getOutputStream());
-
-        } catch (final Exception e) {
-            logger.warn("Failed processing Socket Authorization Provider protocol message due to " + e, e);
-        } finally {
-            SocketUtils.closeQuietly(socket);
-        }
-    }
-
-    @Override
-    @AuthorityProviderContext
-    public void setApplicationContext(final ApplicationContext applicationContext) throws BeansException {
-        this.applicationContext = applicationContext;
-    }
-
-    @Override
-    @AuthorityProviderContext
-    public void setNiFiProperties(NiFiProperties properties) {
-        super.setNiFiProperties(properties);
-        this.properties = properties;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/NodeAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/NodeAuthorizationProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/NodeAuthorizationProvider.java
deleted file mode 100644
index 840422f..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/NodeAuthorizationProvider.java
+++ /dev/null
@@ -1,389 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.cluster.authorization;
-
-import org.apache.nifi.cluster.authorization.protocol.message.DoesDnExistMessage;
-import org.apache.nifi.cluster.authorization.protocol.message.GetAuthoritiesMessage;
-import org.apache.nifi.cluster.authorization.protocol.message.ProtocolMessage;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import org.apache.nifi.authorization.Authority;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.authorization.AuthorityProviderConfigurationContext;
-import org.apache.nifi.authorization.AuthorityProviderInitializationContext;
-import org.apache.nifi.authorization.DownloadAuthorization;
-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.ProviderDestructionException;
-import org.apache.nifi.authorization.exception.UnknownIdentityException;
-import org.apache.nifi.cluster.authorization.protocol.message.GetGroupForUserMessage;
-import org.apache.nifi.cluster.authorization.protocol.message.jaxb.JaxbProtocolUtils;
-import org.apache.nifi.io.socket.SocketConfiguration;
-import org.apache.nifi.io.socket.SocketUtils;
-import org.apache.nifi.io.socket.multicast.DiscoverableService;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
-import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller;
-import org.apache.nifi.cluster.protocol.impl.ClusterServiceDiscovery;
-import org.apache.nifi.cluster.protocol.impl.ClusterServiceLocator;
-import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
-import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
-import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
-import org.apache.nifi.logging.NiFiLog;
-import org.apache.nifi.util.NiFiProperties;
-import static org.apache.nifi.util.NiFiProperties.CLUSTER_NODE_UNICAST_MANAGER_ADDRESS;
-import org.apache.nifi.util.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.springframework.beans.BeansException;
-import org.springframework.context.ApplicationContext;
-import org.springframework.context.ApplicationContextAware;
-
-/**
- * Provides authorities for nodes in clustered environments. Communication
- * occurs over TCP/IP sockets. All method calls are communicated to the cluster
- * manager provider via socket.
- */
-public class NodeAuthorizationProvider implements AuthorityProvider, ApplicationContextAware {
-
-    private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(NodeAuthorizationProvider.class));
-    private static final String CLUSTER_NODE_MANAGER_AUTHORITY_PROVIDER_PORT = "Cluster Manager Authority Provider Port";
-
-    private ProtocolContext<ProtocolMessage> authorityProviderProtocolContext;
-    private SocketConfiguration socketConfiguration;
-    private ClusterServiceLocator serviceLocator;
-    private ApplicationContext applicationContext;
-    private NiFiProperties properties;
-
-    @Override
-    public void initialize(AuthorityProviderInitializationContext initializationContext) throws ProviderCreationException {
-    }
-
-    @Override
-    public void onConfigured(final AuthorityProviderConfigurationContext configurationContext) throws ProviderCreationException {
-        // TODO clear user cache?
-
-        // if using multicast, then the authority provider's service is broadcasted
-        if (properties.getClusterProtocolUseMulticast()) {
-            // create the service discovery
-            final ClusterServiceDiscovery serviceDiscovery = new ClusterServiceDiscovery(
-                    ClusterManagerAuthorizationProvider.AUTHORITY_PROVIDER_SERVIVE_NAME,
-                    properties.getClusterProtocolMulticastAddress(),
-                    applicationContext.getBean("protocolMulticastConfiguration", MulticastConfiguration.class),
-                    applicationContext.getBean("protocolContext", ProtocolContext.class));
-
-            // create service location configuration
-            final ClusterServiceLocator.AttemptsConfig config = new ClusterServiceLocator.AttemptsConfig();
-            config.setNumAttempts(3);
-            config.setTimeBetweenAttempts(1);
-            config.setTimeBetweenAttempsUnit(TimeUnit.SECONDS);
-
-            serviceLocator = new ClusterServiceLocator(serviceDiscovery);
-            serviceLocator.setAttemptsConfig(config);
-        } else {
-            final InetSocketAddress serviceAddress = getClusterNodeManagerAuthorityProviderAddress(configurationContext);
-            final DiscoverableService service = new DiscoverableServiceImpl(ClusterManagerAuthorizationProvider.AUTHORITY_PROVIDER_SERVIVE_NAME, serviceAddress);
-            serviceLocator = new ClusterServiceLocator(service);
-        }
-
-        try {
-            // start the service locator
-            serviceLocator.start();
-        } catch (final IOException ioe) {
-            throw new ProviderCreationException(ioe);
-        }
-
-        // the socket configuration
-        socketConfiguration = applicationContext.getBean("protocolSocketConfiguration", SocketConfiguration.class);
-
-        // initialize the protocol context
-        authorityProviderProtocolContext = new JaxbProtocolContext<ProtocolMessage>(JaxbProtocolUtils.JAXB_CONTEXT);
-    }
-
-    private InetSocketAddress getClusterNodeManagerAuthorityProviderAddress(final AuthorityProviderConfigurationContext configurationContext) {
-        try {
-            String socketAddress = properties.getProperty(CLUSTER_NODE_UNICAST_MANAGER_ADDRESS);
-            if (StringUtils.isBlank(socketAddress)) {
-                socketAddress = "localhost";
-            }
-            return InetSocketAddress.createUnresolved(socketAddress, getClusterNodeManagerAuthorityProviderPort(configurationContext));
-        } catch (Exception ex) {
-            throw new ProviderCreationException("Invalid cluster manager authority provider address/port due to: " + ex, ex);
-        }
-    }
-
-    private Integer getClusterNodeManagerAuthorityProviderPort(final AuthorityProviderConfigurationContext configurationContext) {
-        final String nodeAuthorityProviderPort = configurationContext.getProperty(CLUSTER_NODE_MANAGER_AUTHORITY_PROVIDER_PORT);
-        if (nodeAuthorityProviderPort == null || nodeAuthorityProviderPort.trim().isEmpty()) {
-            throw new ProviderCreationException("The cluster manager authority provider port must be specified.");
-        }
-
-        return Integer.parseInt(nodeAuthorityProviderPort);
-    }
-
-    @Override
-    public void setAuthorities(String dn, Set<Authority> authorities) throws AuthorityAccessException {
-        throw new AuthorityAccessException("Nodes are not allowed to set user authorities.");
-    }
-
-    @Override
-    public void addUser(String dn, String group) throws IdentityAlreadyExistsException, AuthorityAccessException {
-        throw new AuthorityAccessException("Nodes are not allowed to add users.");
-    }
-
-    @Override
-    public boolean doesDnExist(String dn) throws AuthorityAccessException {
-        // create message
-        final DoesDnExistMessage msg = new DoesDnExistMessage();
-        msg.setDn(dn);
-
-        Socket socket = null;
-        try {
-
-            final InetSocketAddress socketAddress = getServiceAddress();
-            if (socketAddress == null) {
-                throw new AuthorityAccessException("Cluster Authority Provider's address is not known.");
-            }
-
-            try {
-                // create a socket
-                socket = SocketUtils.createSocket(socketAddress, socketConfiguration);
-            } catch (final IOException ioe) {
-                throw new AuthorityAccessException("Failed to create socket due to: " + ioe, ioe);
-            }
-
-            try {
-                // marshal message to output stream
-                final ProtocolMessageMarshaller marshaller = authorityProviderProtocolContext.createMarshaller();
-                marshaller.marshal(msg, socket.getOutputStream());
-            } catch (final IOException ioe) {
-                throw new AuthorityAccessException("Failed marshalling '" + msg.getType() + "' protocol message due to: " + ioe, ioe);
-            }
-
-            try {
-
-                // unmarshall response and return
-                final ProtocolMessageUnmarshaller<ProtocolMessage> unmarshaller = authorityProviderProtocolContext.createUnmarshaller();
-                final DoesDnExistMessage response = (DoesDnExistMessage) unmarshaller.unmarshal(socket.getInputStream());
-
-                // check if there was an exception
-                if (response.wasException()) {
-                    throw new AuthorityAccessException(response.getExceptionMessage());
-                }
-
-                // return provider's response
-                return response.getResponse();
-
-            } catch (final IOException ioe) {
-                throw new AuthorityAccessException("Failed unmarshalling '" + msg.getType() + "' response protocol message due to: " + ioe, ioe);
-            }
-
-        } finally {
-            SocketUtils.closeQuietly(socket);
-        }
-    }
-
-    @Override
-    public Set<Authority> getAuthorities(String dn) throws UnknownIdentityException, AuthorityAccessException {
-        // create message
-        final GetAuthoritiesMessage msg = new GetAuthoritiesMessage();
-        msg.setDn(dn);
-
-        Socket socket = null;
-        try {
-
-            final InetSocketAddress socketAddress = getServiceAddress();
-            if (socketAddress == null) {
-                throw new AuthorityAccessException("Cluster Authority Provider's address is not known.");
-            }
-
-            try {
-                // create a socket
-                socket = SocketUtils.createSocket(socketAddress, socketConfiguration);
-            } catch (final IOException ioe) {
-                throw new AuthorityAccessException("Failed to create socket due to: " + ioe, ioe);
-            }
-
-            try {
-                // marshal message to output stream
-                final ProtocolMessageMarshaller marshaller = authorityProviderProtocolContext.createMarshaller();
-                marshaller.marshal(msg, socket.getOutputStream());
-            } catch (final IOException ioe) {
-                throw new AuthorityAccessException("Failed marshalling '" + msg.getType() + "' protocol message due to: " + ioe, ioe);
-            }
-
-            try {
-
-                // unmarshall response and return
-                final ProtocolMessageUnmarshaller<ProtocolMessage> unmarshaller = authorityProviderProtocolContext.createUnmarshaller();
-                final GetAuthoritiesMessage response = (GetAuthoritiesMessage) unmarshaller.unmarshal(socket.getInputStream());
-
-                // check if there was an exception
-                if (response.wasException()) {
-                    if (isException(UnknownIdentityException.class, response)) {
-                        throw new UnknownIdentityException(response.getExceptionMessage());
-                    } else {
-                        throw new AuthorityAccessException(response.getExceptionMessage());
-                    }
-                }
-
-                // return provider's response
-                return response.getResponse();
-
-            } catch (final IOException ioe) {
-                throw new AuthorityAccessException("Failed unmarshalling '" + msg.getType() + "' response protocol message due to: " + ioe, ioe);
-            }
-
-        } finally {
-            SocketUtils.closeQuietly(socket);
-        }
-    }
-
-    @Override
-    public Set<String> getUsers(Authority authority) throws AuthorityAccessException {
-        throw new AuthorityAccessException("Nodes are not allowed to get users for a given authority.");
-    }
-
-    @Override
-    public void revokeUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
-        throw new AuthorityAccessException("Nodes are not allowed to revoke users.");
-    }
-
-    @Override
-    public void setUsersGroup(Set<String> dns, String group) throws UnknownIdentityException, AuthorityAccessException {
-        throw new AuthorityAccessException("Nodes are not allowed to set user groups.");
-    }
-
-    @Override
-    public void ungroupUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
-        throw new AuthorityAccessException("Nodes are not allowed to ungroup users.");
-    }
-
-    @Override
-    public void ungroup(String group) throws AuthorityAccessException {
-        throw new AuthorityAccessException("Nodes are not allowed to ungroup.");
-    }
-
-    @Override
-    public DownloadAuthorization authorizeDownload(List<String> dnChain, Map<String, String> attributes) throws UnknownIdentityException, AuthorityAccessException {
-        return DownloadAuthorization.approved();
-    }
-
-    @Override
-    public String getGroupForUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
-        // create message
-        final GetGroupForUserMessage msg = new GetGroupForUserMessage();
-        msg.setDn(dn);
-
-        Socket socket = null;
-        try {
-
-            final InetSocketAddress socketAddress = getServiceAddress();
-            if (socketAddress == null) {
-                throw new AuthorityAccessException("Cluster Authority Provider's address is not known.");
-            }
-
-            try {
-                // create a socket
-                socket = SocketUtils.createSocket(socketAddress, socketConfiguration);
-            } catch (final IOException ioe) {
-                throw new AuthorityAccessException("Failed to create socket due to: " + ioe, ioe);
-            }
-
-            try {
-                // marshal message to output stream
-                final ProtocolMessageMarshaller marshaller = authorityProviderProtocolContext.createMarshaller();
-                marshaller.marshal(msg, socket.getOutputStream());
-            } catch (final IOException ioe) {
-                throw new AuthorityAccessException("Failed marshalling '" + msg.getType() + "' protocol message due to: " + ioe, ioe);
-            }
-
-            try {
-
-                // unmarshall response and return
-                final ProtocolMessageUnmarshaller<ProtocolMessage> unmarshaller = authorityProviderProtocolContext.createUnmarshaller();
-                final GetGroupForUserMessage response = (GetGroupForUserMessage) unmarshaller.unmarshal(socket.getInputStream());
-
-                // check if there was an exception
-                if (response.wasException()) {
-                    if (isException(UnknownIdentityException.class, response)) {
-                        throw new UnknownIdentityException(response.getExceptionMessage());
-                    } else {
-                        throw new AuthorityAccessException(response.getExceptionMessage());
-                    }
-                }
-
-                return response.getResponse();
-            } catch (final IOException ioe) {
-                throw new AuthorityAccessException("Failed unmarshalling '" + msg.getType() + "' response protocol message due to: " + ioe, ioe);
-            }
-
-        } finally {
-            SocketUtils.closeQuietly(socket);
-        }
-    }
-
-    @Override
-    public void revokeGroup(String group) throws UnknownIdentityException, AuthorityAccessException {
-        throw new AuthorityAccessException("Nodes are not allowed to revoke groups.");
-    }
-
-    @Override
-    public void preDestruction() throws ProviderDestructionException {
-        try {
-            if (serviceLocator != null && serviceLocator.isRunning()) {
-                serviceLocator.stop();
-            }
-        } catch (final IOException ioe) {
-            throw new ProviderDestructionException(ioe);
-        }
-    }
-
-    @Override
-    @AuthorityProviderContext
-    public void setApplicationContext(final ApplicationContext applicationContext) throws BeansException {
-        this.applicationContext = applicationContext;
-    }
-
-    @AuthorityProviderContext
-    public void setNiFiProperties(NiFiProperties properties) {
-        this.properties = properties;
-    }
-
-    private InetSocketAddress getServiceAddress() {
-        final DiscoverableService service = serviceLocator.getService();
-        if (service != null) {
-            return service.getServiceAddress();
-        }
-        return null;
-    }
-
-    private boolean isException(final Class<? extends Exception> exception, final ProtocolMessage protocolMessage) {
-        if (protocolMessage.wasException()) {
-            return exception.getName().equals(protocolMessage.getExceptionClass());
-        } else {
-            return false;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/DoesDnExistMessage.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/DoesDnExistMessage.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/DoesDnExistMessage.java
deleted file mode 100644
index 5436140..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/DoesDnExistMessage.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.cluster.authorization.protocol.message;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.cluster.authorization.protocol.message.ProtocolMessage.MessageType;
-
-/**
- */
-@XmlRootElement(name = "doesDnExistMessage")
-public class DoesDnExistMessage extends ProtocolMessage {
-
-    private String dn;
-
-    private boolean response;
-
-    public DoesDnExistMessage() {
-    }
-
-    @Override
-    public MessageType getType() {
-        return MessageType.DOES_DN_EXIST;
-    }
-
-    public String getDn() {
-        return dn;
-    }
-
-    public void setDn(String dn) {
-        this.dn = dn;
-    }
-
-    public boolean getResponse() {
-        return response;
-    }
-
-    public void setResponse(boolean response) {
-        this.response = response;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/GetAuthoritiesMessage.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/GetAuthoritiesMessage.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/GetAuthoritiesMessage.java
deleted file mode 100644
index 50d371d..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/GetAuthoritiesMessage.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.cluster.authorization.protocol.message;
-
-import java.util.HashSet;
-import java.util.Set;
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.authorization.Authority;
-
-/**
- */
-@XmlRootElement(name = "getAuthoritiesMessage")
-public class GetAuthoritiesMessage extends ProtocolMessage {
-
-    private String dn;
-
-    private Set<Authority> response = new HashSet<>();
-
-    public GetAuthoritiesMessage() {
-    }
-
-    @Override
-    public MessageType getType() {
-        return MessageType.GET_AUTHORITIES;
-    }
-
-    public String getDn() {
-        return dn;
-    }
-
-    public void setDn(String dn) {
-        this.dn = dn;
-    }
-
-    public Set<Authority> getResponse() {
-        return response;
-    }
-
-    public void setResponse(Set<Authority> response) {
-        this.response = response;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/GetGroupForUserMessage.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/GetGroupForUserMessage.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/GetGroupForUserMessage.java
deleted file mode 100644
index 72a6af5..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/GetGroupForUserMessage.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.cluster.authorization.protocol.message;
-
-import javax.xml.bind.annotation.XmlRootElement;
-
-/**
- */
-@XmlRootElement(name = "getGroupForUserMessage")
-public class GetGroupForUserMessage extends ProtocolMessage {
-
-    private String dn;
-
-    private String response;
-
-    public GetGroupForUserMessage() {
-    }
-
-    @Override
-    public MessageType getType() {
-        return MessageType.GET_GROUP_FOR_USER;
-    }
-
-    public String getDn() {
-        return dn;
-    }
-
-    public void setDn(String dn) {
-        this.dn = dn;
-    }
-
-    public String getResponse() {
-        return response;
-    }
-
-    public void setResponse(String response) {
-        this.response = response;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/ProtocolMessage.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/ProtocolMessage.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/ProtocolMessage.java
deleted file mode 100644
index ddeb69e..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/ProtocolMessage.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.cluster.authorization.protocol.message;
-
-/**
- */
-public abstract class ProtocolMessage {
-
-    private String exceptionClass;
-    private String exceptionMessage;
-
-    public static enum MessageType {
-
-        DOES_DN_EXIST,
-        GET_AUTHORITIES,
-        GET_USERS,
-        GET_GROUP_FOR_USER
-    }
-
-    public abstract MessageType getType();
-
-    public boolean wasException() {
-        return exceptionClass != null;
-    }
-
-    public String getExceptionMessage() {
-        return exceptionMessage;
-    }
-
-    public void setExceptionMessage(final String exceptionMessage) {
-        this.exceptionMessage = exceptionMessage;
-    }
-
-    public String getExceptionClass() {
-        return exceptionClass;
-    }
-
-    public void setExceptionClass(String exceptionClass) {
-        this.exceptionClass = exceptionClass;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/jaxb/JaxbProtocolUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/jaxb/JaxbProtocolUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/jaxb/JaxbProtocolUtils.java
deleted file mode 100644
index 2a32d84..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/jaxb/JaxbProtocolUtils.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.cluster.authorization.protocol.message.jaxb;
-
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBException;
-
-/**
- */
-public final class JaxbProtocolUtils {
-
-    public static final String JAXB_CONTEXT_PATH = ObjectFactory.class.getPackage().getName();
-
-    public static final JAXBContext JAXB_CONTEXT = initializeJaxbContext();
-
-    /**
-     * Load the JAXBContext version.
-     */
-    private static JAXBContext initializeJaxbContext() {
-        try {
-            return JAXBContext.newInstance(JAXB_CONTEXT_PATH);
-        } catch (JAXBException e) {
-            throw new RuntimeException("Unable to create JAXBContext.");
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/jaxb/ObjectFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/jaxb/ObjectFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/jaxb/ObjectFactory.java
deleted file mode 100644
index 2e70a19..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/jaxb/ObjectFactory.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.cluster.authorization.protocol.message.jaxb;
-
-import javax.xml.bind.annotation.XmlRegistry;
-import org.apache.nifi.cluster.authorization.protocol.message.DoesDnExistMessage;
-import org.apache.nifi.cluster.authorization.protocol.message.GetAuthoritiesMessage;
-import org.apache.nifi.cluster.authorization.protocol.message.GetGroupForUserMessage;
-
-/**
- */
-@XmlRegistry
-public class ObjectFactory {
-
-    public ObjectFactory() {
-    }
-
-    public DoesDnExistMessage createDoesDnExistMessage() {
-        return new DoesDnExistMessage();
-    }
-
-    public GetAuthoritiesMessage createGetAuthoritiesMessage() {
-        return new GetAuthoritiesMessage();
-    }
-
-    public GetGroupForUserMessage createGetGroupForUserMessage() {
-        return new GetGroupForUserMessage();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-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-cluster-authorization-provider/src/main/resources/META-INF/services/org.apache.nifi.authorization.AuthorityProvider b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/resources/META-INF/services/org.apache.nifi.authorization.AuthorityProvider
deleted file mode 100644
index 56f4c3e..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/src/main/resources/META-INF/services/org.apache.nifi.authorization.AuthorityProvider
+++ /dev/null
@@ -1,16 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-org.apache.nifi.cluster.authorization.ClusterManagerAuthorizationProvider
-org.apache.nifi.cluster.authorization.NodeAuthorizationProvider
\ 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-documentation/src/test/resources/conf/nifi.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/resources/conf/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/resources/conf/nifi.properties
index bfc9376..c6b5d36 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/resources/conf/nifi.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/resources/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-file-authorization-provider/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/pom.xml
deleted file mode 100644
index caa75de..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/pom.xml
+++ /dev/null
@@ -1,85 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-framework</artifactId>
-        <version>1.0.0-SNAPSHOT</version>
-    </parent>
-    <artifactId>nifi-file-authorization-provider</artifactId>
-    <build>
-        <resources>
-            <resource>
-                <directory>src/main/resources</directory>
-            </resource>
-            <resource>
-                <directory>src/main/xsd</directory>
-            </resource>
-        </resources>
-        <plugins>
-            <plugin>
-                <groupId>org.codehaus.mojo</groupId>
-                <artifactId>jaxb2-maven-plugin</artifactId>
-                <executions>
-                    <execution>
-                        <id>xjc</id>
-                        <goals>
-                            <goal>xjc</goal>
-                        </goals>
-                        <configuration>
-                            <packageName>org.apache.nifi.user.generated</packageName>
-                        </configuration>
-                    </execution>
-                </executions>
-                <configuration>
-                    <generateDirectory>${project.build.directory}/generated-sources/jaxb</generateDirectory>
-                </configuration>
-            </plugin>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-checkstyle-plugin</artifactId>
-                <configuration>
-                    <excludes>**/user/generated/*.java</excludes>
-                </configuration>
-            </plugin>            
-
-        </plugins>
-    </build>
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-api</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-utils</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-properties</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.commons</groupId>
-            <artifactId>commons-lang3</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>commons-codec</groupId>
-            <artifactId>commons-codec</artifactId>
-            <scope>test</scope>
-        </dependency>
-    </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/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
deleted file mode 100644
index 9c2cad5..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/src/main/java/org/apache/nifi/authorization/FileAuthorizationProvider.java
+++ /dev/null
@@ -1,496 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.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/153f63ef/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
deleted file mode 100755
index 93d2941..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/src/main/resources/META-INF/services/org.apache.nifi.authorization.AuthorityProvider
+++ /dev/null
@@ -1,15 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-org.apache.nifi.authorization.FileAuthorizationProvider

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/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
deleted file mode 100644
index 4ee1e17..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/src/main/xsd/users.xsd
+++ /dev/null
@@ -1,64 +0,0 @@
-<?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/153f63ef/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
deleted file mode 100644
index 7428500..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/src/test/java/org/apache/nifi/authorization/FileAuthorizationProviderTest.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.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/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileAuthorizer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileAuthorizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileAuthorizer.java
index 174e501..8529caf 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileAuthorizer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileAuthorizer.java
@@ -19,7 +19,7 @@ package org.apache.nifi.authorization;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.annotation.AuthorizerContext;
 import org.apache.nifi.authorization.exception.AuthorizationAccessException;
-import org.apache.nifi.authorization.exception.ProviderCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
 import org.apache.nifi.authorization.generated.Authorization;
 import org.apache.nifi.authorization.generated.Resource;
 import org.apache.nifi.authorization.generated.Resources;
@@ -85,21 +85,21 @@ public class FileAuthorizer implements Authorizer {
     private final AtomicReference<Map<String, Map<String, Set<RequestAction>>>> authorizations = new AtomicReference<>();
 
     @Override
-    public void initialize(final AuthorizerInitializationContext initializationContext) throws ProviderCreationException {
+    public void initialize(final AuthorizerInitializationContext initializationContext) throws AuthorizerCreationException {
     }
 
     @Override
-    public void onConfigured(final AuthorizerConfigurationContext configurationContext) throws ProviderCreationException {
+    public void onConfigured(final AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
         try {
             final PropertyValue authorizationsPath = configurationContext.getProperty("Authorizations File");
             if (StringUtils.isBlank(authorizationsPath.getValue())) {
-                throw new ProviderCreationException("The authorizations file must be specified.");
+                throw new AuthorizerCreationException("The authorizations file must be specified.");
             }
 
             // get the authorizations file and ensure it exists
             authorizationsFile = new File(authorizationsPath.getValue());
             if (!authorizationsFile.exists()) {
-                throw new ProviderCreationException("The authorizations file must exist.");
+                throw new AuthorizerCreationException("The authorizations file must exist.");
             }
 
             final File authorizationsFileDirectory = authorizationsFile.getAbsoluteFile().getParentFile();
@@ -112,7 +112,7 @@ public class FileAuthorizer implements Authorizer {
 
                 // check that restore directory is not the same as the primary directory
                 if (authorizationsFileDirectory.getAbsolutePath().equals(restoreDirectory.getAbsolutePath())) {
-                    throw new ProviderCreationException(String.format("Authorizations file directory '%s' is the same as restore directory '%s' ",
+                    throw new AuthorizerCreationException(String.format("Authorizations file directory '%s' is the same as restore directory '%s' ",
                             authorizationsFileDirectory.getAbsolutePath(), restoreDirectory.getAbsolutePath()));
                 }
 
@@ -123,7 +123,7 @@ public class FileAuthorizer implements Authorizer {
                     // sync the primary copy with the restore copy
                     FileUtils.syncWithRestore(authorizationsFile, restoreAuthorizationsFile, logger);
                 } catch (final IOException | IllegalStateException ioe) {
-                    throw new ProviderCreationException(ioe);
+                    throw new AuthorizerCreationException(ioe);
                 }
             }
 
@@ -160,8 +160,8 @@ public class FileAuthorizer implements Authorizer {
                     }
                 }
             }, reloadInterval, reloadInterval, TimeUnit.MILLISECONDS);
-        } catch (IOException | ProviderCreationException | SAXException | JAXBException | IllegalStateException e) {
-            throw new ProviderCreationException(e);
+        } catch (IOException | AuthorizerCreationException | SAXException | JAXBException | IllegalStateException e) {
+            throw new AuthorizerCreationException(e);
         }
 
     }