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:19:18 UTC

[6/9] nifi git commit: NIFI-1551: - Starting to remove the AuthorityProvider. - This closes #330

http://git-wip-us.apache.org/repos/asf/nifi/blob/c4d06f20/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/authorization/AuthorityProviderFactoryBean.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/authorization/AuthorityProviderFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/authorization/AuthorityProviderFactoryBean.java
deleted file mode 100644
index e1a02b8..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/authorization/AuthorityProviderFactoryBean.java
+++ /dev/null
@@ -1,491 +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 org.apache.commons.lang3.StringUtils;
-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.authorization.generated.AuthorityProviderProperty;
-import org.apache.nifi.authorization.generated.AuthorityProviders;
-import org.apache.nifi.authorization.generated.Provider;
-import org.apache.nifi.nar.ExtensionManager;
-import org.apache.nifi.nar.NarCloseable;
-import org.apache.nifi.util.NiFiProperties;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.springframework.beans.BeansException;
-import org.springframework.beans.factory.DisposableBean;
-import org.springframework.beans.factory.FactoryBean;
-import org.springframework.context.ApplicationContext;
-import org.springframework.context.ApplicationContextAware;
-import org.xml.sax.SAXException;
-
-import javax.xml.XMLConstants;
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBElement;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Unmarshaller;
-import javax.xml.transform.stream.StreamSource;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
-import java.io.File;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.Field;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Factory bean for loading the configured authority provider.
- */
-public class AuthorityProviderFactoryBean implements FactoryBean, ApplicationContextAware, DisposableBean, AuthorityProviderLookup {
-
-    private static final Logger logger = LoggerFactory.getLogger(AuthorityProviderFactoryBean.class);
-    private static final String AUTHORITY_PROVIDERS_XSD = "/authority-providers.xsd";
-    private static final String JAXB_GENERATED_PATH = "org.apache.nifi.authorization.generated";
-    private static final JAXBContext JAXB_CONTEXT = initializeJaxbContext();
-
-    /**
-     * Load the JAXBContext.
-     */
-    private static JAXBContext initializeJaxbContext() {
-        try {
-            return JAXBContext.newInstance(JAXB_GENERATED_PATH, AuthorityProviderFactoryBean.class.getClassLoader());
-        } catch (JAXBException e) {
-            throw new RuntimeException("Unable to create JAXBContext.");
-        }
-    }
-
-    private ApplicationContext applicationContext;
-    private AuthorityProvider authorityProvider;
-    private NiFiProperties properties;
-    private final Map<String, AuthorityProvider> authorityProviders = new HashMap<>();
-
-    @Override
-    public AuthorityProvider getAuthorityProvider(String identifier) {
-        return authorityProviders.get(identifier);
-    }
-
-    @Override
-    public Object getObject() throws Exception {
-        if (authorityProvider == null) {
-            // look up the authority provider to use
-            final String authorityProviderIdentifier = properties.getProperty(NiFiProperties.SECURITY_USER_AUTHORITY_PROVIDER);
-
-            // ensure the authority provider class name was specified
-            if (StringUtils.isBlank(authorityProviderIdentifier)) {
-                // if configured for ssl, the authority provider must be specified
-                if (properties.getSslPort() != null) {
-                    throw new Exception("When running securely, the authority provider identifier must be specified in the nifi properties file.");
-                }
-
-                // use a default provider... only allowable when running not securely
-                authorityProvider = createDefaultProvider();
-            } else {
-                final AuthorityProviders authorityProviderConfiguration = loadAuthorityProvidersConfiguration();
-
-                // create each authority provider
-                for (final Provider provider : authorityProviderConfiguration.getProvider()) {
-                    authorityProviders.put(provider.getIdentifier(), createAuthorityProvider(provider.getIdentifier(), provider.getClazz()));
-                }
-
-                // configure each authority provider
-                for (final Provider provider : authorityProviderConfiguration.getProvider()) {
-                    final AuthorityProvider instance = authorityProviders.get(provider.getIdentifier());
-                    instance.onConfigured(loadAuthorityProviderConfiguration(provider));
-                }
-
-                // get the authority provider instance
-                authorityProvider = getAuthorityProvider(authorityProviderIdentifier);
-
-                // ensure it was found
-                if (authorityProvider == null) {
-                    throw new Exception(String.format("The specified authority provider '%s' could not be found.", authorityProviderIdentifier));
-                }
-            }
-        }
-
-        return authorityProvider;
-    }
-
-    private AuthorityProviders loadAuthorityProvidersConfiguration() throws Exception {
-        final File authorityProvidersConfigurationFile = properties.getAuthorityProviderConfiguraitonFile();
-
-        // load the users from the specified file
-        if (authorityProvidersConfigurationFile.exists()) {
-            try {
-                // find the schema
-                final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
-                final Schema schema = schemaFactory.newSchema(AuthorityProviders.class.getResource(AUTHORITY_PROVIDERS_XSD));
-
-                // attempt to unmarshal
-                final Unmarshaller unmarshaller = JAXB_CONTEXT.createUnmarshaller();
-                unmarshaller.setSchema(schema);
-                final JAXBElement<AuthorityProviders> element = unmarshaller.unmarshal(new StreamSource(authorityProvidersConfigurationFile), AuthorityProviders.class);
-                return element.getValue();
-            } catch (SAXException | JAXBException e) {
-                throw new Exception("Unable to load the authority provider configuration file at: " + authorityProvidersConfigurationFile.getAbsolutePath());
-            }
-        } else {
-            throw new Exception("Unable to find the authority provider configuration file at " + authorityProvidersConfigurationFile.getAbsolutePath());
-        }
-    }
-
-    private AuthorityProvider createAuthorityProvider(final String identifier, final String authorityProviderClassName) throws Exception {
-        // get the classloader for the specified authority provider
-        final ClassLoader authorityProviderClassLoader = ExtensionManager.getClassLoader(authorityProviderClassName);
-        if (authorityProviderClassLoader == null) {
-            throw new Exception(String.format("The specified authority provider class '%s' is not known to this nifi.", authorityProviderClassName));
-        }
-
-        // get the current context classloader
-        final ClassLoader currentClassLoader = Thread.currentThread().getContextClassLoader();
-
-        final AuthorityProvider instance;
-        try {
-            // set the appropriate class loader
-            Thread.currentThread().setContextClassLoader(authorityProviderClassLoader);
-
-            // attempt to load the class
-            Class<?> rawAuthorityProviderClass = Class.forName(authorityProviderClassName, true, authorityProviderClassLoader);
-            Class<? extends AuthorityProvider> authorityProviderClass = rawAuthorityProviderClass.asSubclass(AuthorityProvider.class);
-
-            // otherwise create a new instance
-            Constructor constructor = authorityProviderClass.getConstructor();
-            instance = (AuthorityProvider) constructor.newInstance();
-
-            // method injection
-            performMethodInjection(instance, authorityProviderClass);
-
-            // field injection
-            performFieldInjection(instance, authorityProviderClass);
-
-            // call post construction lifecycle event
-            instance.initialize(new StandardAuthorityProviderInitializationContext(identifier, this));
-        } finally {
-            if (currentClassLoader != null) {
-                Thread.currentThread().setContextClassLoader(currentClassLoader);
-            }
-        }
-
-        return withNarLoader(instance);
-    }
-
-    private AuthorityProviderConfigurationContext loadAuthorityProviderConfiguration(final Provider provider) {
-        final Map<String, String> providerProperties = new HashMap<>();
-
-        for (final AuthorityProviderProperty property : provider.getProperty()) {
-            providerProperties.put(property.getName(), property.getValue());
-        }
-
-        return new StandardAuthorityProviderConfigurationContext(provider.getIdentifier(), providerProperties);
-    }
-
-    private void performMethodInjection(final AuthorityProvider instance, final Class authorityProviderClass) throws IllegalAccessException, IllegalArgumentException, InvocationTargetException {
-        for (final Method method : authorityProviderClass.getMethods()) {
-            if (method.isAnnotationPresent(AuthorityProviderContext.class)) {
-                // make the method accessible
-                final boolean isAccessible = method.isAccessible();
-                method.setAccessible(true);
-
-                try {
-                    final Class<?>[] argumentTypes = method.getParameterTypes();
-
-                    // look for setters (single argument)
-                    if (argumentTypes.length == 1) {
-                        final Class<?> argumentType = argumentTypes[0];
-
-                        // look for well known types
-                        if (NiFiProperties.class.isAssignableFrom(argumentType)) {
-                            // nifi properties injection
-                            method.invoke(instance, properties);
-                        } else if (ApplicationContext.class.isAssignableFrom(argumentType)) {
-                            // spring application context injection
-                            method.invoke(instance, applicationContext);
-                        }
-                    }
-                } finally {
-                    method.setAccessible(isAccessible);
-                }
-            }
-        }
-
-        final Class parentClass = authorityProviderClass.getSuperclass();
-        if (parentClass != null && AuthorityProvider.class.isAssignableFrom(parentClass)) {
-            performMethodInjection(instance, parentClass);
-        }
-    }
-
-    private void performFieldInjection(final AuthorityProvider instance, final Class authorityProviderClass) throws IllegalArgumentException, IllegalAccessException {
-        for (final Field field : authorityProviderClass.getDeclaredFields()) {
-            if (field.isAnnotationPresent(AuthorityProviderContext.class)) {
-                // make the method accessible
-                final boolean isAccessible = field.isAccessible();
-                field.setAccessible(true);
-
-                try {
-                    // get the type
-                    final Class<?> fieldType = field.getType();
-
-                    // only consider this field if it isn't set yet
-                    if (field.get(instance) == null) {
-                        // look for well known types
-                        if (NiFiProperties.class.isAssignableFrom(fieldType)) {
-                            // nifi properties injection
-                            field.set(instance, properties);
-                        } else if (ApplicationContext.class.isAssignableFrom(fieldType)) {
-                            // spring application context injection
-                            field.set(instance, applicationContext);
-                        }
-                    }
-
-                } finally {
-                    field.setAccessible(isAccessible);
-                }
-            }
-        }
-
-        final Class parentClass = authorityProviderClass.getSuperclass();
-        if (parentClass != null && AuthorityProvider.class.isAssignableFrom(parentClass)) {
-            performFieldInjection(instance, parentClass);
-        }
-    }
-
-    /**
-     * @return a default provider to use when running unsecurely with no
-     * provider configured
-     */
-    private AuthorityProvider createDefaultProvider() {
-        return new AuthorityProvider() {
-            @Override
-            public boolean doesDnExist(String dn) throws AuthorityAccessException {
-                return false;
-            }
-
-            @Override
-            public Set<Authority> getAuthorities(String dn) throws UnknownIdentityException, AuthorityAccessException {
-                return EnumSet.noneOf(Authority.class);
-            }
-
-            @Override
-            public void setAuthorities(String dn, Set<Authority> authorities) throws UnknownIdentityException, AuthorityAccessException {
-            }
-
-            @Override
-            public Set<String> getUsers(Authority authority) throws AuthorityAccessException {
-                return new HashSet<>();
-            }
-
-            @Override
-            public void revokeUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
-            }
-
-            @Override
-            public void addUser(String dn, String group) throws IdentityAlreadyExistsException, AuthorityAccessException {
-            }
-
-            @Override
-            public String getGroupForUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
-                return null;
-            }
-
-            @Override
-            public void revokeGroup(String group) throws UnknownIdentityException, AuthorityAccessException {
-            }
-
-            @Override
-            public void setUsersGroup(Set<String> dn, String group) throws UnknownIdentityException, AuthorityAccessException {
-            }
-
-            @Override
-            public void ungroupUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
-            }
-
-            @Override
-            public void ungroup(String group) throws AuthorityAccessException {
-            }
-
-            @Override
-            public DownloadAuthorization authorizeDownload(List<String> dnChain, Map<String, String> attributes) throws UnknownIdentityException, AuthorityAccessException {
-                return DownloadAuthorization.approved();
-            }
-
-            @Override
-            public void initialize(AuthorityProviderInitializationContext initializationContext) throws ProviderCreationException {
-            }
-
-            @Override
-            public void onConfigured(AuthorityProviderConfigurationContext configurationContext) throws ProviderCreationException {
-            }
-
-            @Override
-            public void preDestruction() throws ProviderDestructionException {
-            }
-        };
-    }
-
-    /**
-     * Decorates the base provider to ensure the nar context classloader is used
-     * when invoking the underlying methods.
-     *
-     * @param baseProvider base provider
-     * @return provider
-     */
-    public AuthorityProvider withNarLoader(final AuthorityProvider baseProvider) {
-        return new AuthorityProvider() {
-            @Override
-            public boolean doesDnExist(String dn) throws AuthorityAccessException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    return baseProvider.doesDnExist(dn);
-                }
-            }
-
-            @Override
-            public Set<Authority> getAuthorities(String dn) throws UnknownIdentityException, AuthorityAccessException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    return baseProvider.getAuthorities(dn);
-                }
-            }
-
-            @Override
-            public void setAuthorities(String dn, Set<Authority> authorities) throws UnknownIdentityException, AuthorityAccessException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    baseProvider.setAuthorities(dn, authorities);
-                }
-            }
-
-            @Override
-            public Set<String> getUsers(Authority authority) throws AuthorityAccessException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    return baseProvider.getUsers(authority);
-                }
-            }
-
-            @Override
-            public void revokeUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    baseProvider.revokeUser(dn);
-                }
-            }
-
-            @Override
-            public void addUser(String dn, String group) throws IdentityAlreadyExistsException, AuthorityAccessException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    baseProvider.addUser(dn, group);
-                }
-            }
-
-            @Override
-            public String getGroupForUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    return baseProvider.getGroupForUser(dn);
-                }
-            }
-
-            @Override
-            public void revokeGroup(String group) throws UnknownIdentityException, AuthorityAccessException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    baseProvider.revokeGroup(group);
-                }
-            }
-
-            @Override
-            public void setUsersGroup(Set<String> dns, String group) throws UnknownIdentityException, AuthorityAccessException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    baseProvider.setUsersGroup(dns, group);
-                }
-            }
-
-            @Override
-            public void ungroupUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    baseProvider.ungroupUser(dn);
-                }
-            }
-
-            @Override
-            public void ungroup(String group) throws AuthorityAccessException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    baseProvider.ungroup(group);
-                }
-            }
-
-            @Override
-            public DownloadAuthorization authorizeDownload(List<String> dnChain, Map<String, String> attributes) throws UnknownIdentityException, AuthorityAccessException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    return baseProvider.authorizeDownload(dnChain, attributes);
-                }
-            }
-
-            @Override
-            public void initialize(AuthorityProviderInitializationContext initializationContext) throws ProviderCreationException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    baseProvider.initialize(initializationContext);
-                }
-            }
-
-            @Override
-            public void onConfigured(AuthorityProviderConfigurationContext configurationContext) throws ProviderCreationException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    baseProvider.onConfigured(configurationContext);
-                }
-            }
-
-            @Override
-            public void preDestruction() throws ProviderDestructionException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    baseProvider.preDestruction();
-                }
-            }
-        };
-    }
-
-    @Override
-    public Class getObjectType() {
-        return AuthorityProvider.class;
-    }
-
-    @Override
-    public boolean isSingleton() {
-        return true;
-    }
-
-    @Override
-    public void setApplicationContext(ApplicationContext applicationContext) throws BeansException {
-        this.applicationContext = applicationContext;
-    }
-
-    @Override
-    public void destroy() throws Exception {
-        if (authorityProvider != null) {
-            authorityProvider.preDestruction();
-        }
-    }
-
-    public void setProperties(NiFiProperties properties) {
-        this.properties = properties;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/c4d06f20/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/authorization/AuthorizerFactoryBean.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/authorization/AuthorizerFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/authorization/AuthorizerFactoryBean.java
index 58caea9..cf35c15 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/authorization/AuthorizerFactoryBean.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/authorization/AuthorizerFactoryBean.java
@@ -21,7 +21,6 @@ import org.apache.nifi.authorization.annotation.AuthorizerContext;
 import org.apache.nifi.authorization.exception.AuthorizationAccessException;
 import org.apache.nifi.authorization.exception.AuthorizerCreationException;
 import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
-import org.apache.nifi.authorization.generated.AuthorityProviders;
 import org.apache.nifi.authorization.generated.Authorizers;
 import org.apache.nifi.authorization.generated.Property;
 import org.apache.nifi.nar.ExtensionManager;
@@ -83,7 +82,7 @@ public class AuthorizerFactoryBean implements FactoryBean, DisposableBean, Autho
     public Object getObject() throws Exception {
         if (authorizer == null) {
             // look up the authorizer to use
-            final String authorizerIdentifier = properties.getProperty(NiFiProperties.SECURITY_USER_AUTHORITY_PROVIDER);
+            final String authorizerIdentifier = properties.getProperty(NiFiProperties.SECURITY_USER_AUTHORIZER);
 
             // ensure the authorizer class name was specified
             if (StringUtils.isBlank(authorizerIdentifier)) {
@@ -122,14 +121,14 @@ public class AuthorizerFactoryBean implements FactoryBean, DisposableBean, Autho
     }
 
     private Authorizers loadAuthorizersConfiguration() throws Exception {
-        final File authorizersConfigurationFile = properties.getAuthorityProviderConfiguraitonFile();
+        final File authorizersConfigurationFile = properties.getAuthorizerConfiguraitonFile();
 
         // load the authorizers from the specified file
         if (authorizersConfigurationFile.exists()) {
             try {
                 // find the schema
                 final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
-                final Schema schema = schemaFactory.newSchema(AuthorityProviders.class.getResource(AUTHORIZERS_XSD));
+                final Schema schema = schemaFactory.newSchema(Authorizers.class.getResource(AUTHORIZERS_XSD));
 
                 // attempt to unmarshal
                 final Unmarshaller unmarshaller = JAXB_CONTEXT.createUnmarshaller();
@@ -221,7 +220,7 @@ public class AuthorizerFactoryBean implements FactoryBean, DisposableBean, Autho
         }
 
         final Class parentClass = authorizerClass.getSuperclass();
-        if (parentClass != null && AuthorityProvider.class.isAssignableFrom(parentClass)) {
+        if (parentClass != null && Authorizer.class.isAssignableFrom(parentClass)) {
             performMethodInjection(instance, parentClass);
         }
     }
@@ -253,7 +252,7 @@ public class AuthorizerFactoryBean implements FactoryBean, DisposableBean, Autho
         }
 
         final Class parentClass = authorizerClass.getSuperclass();
-        if (parentClass != null && AuthorityProvider.class.isAssignableFrom(parentClass)) {
+        if (parentClass != null && Authorizer.class.isAssignableFrom(parentClass)) {
             performFieldInjection(instance, parentClass);
         }
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/c4d06f20/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/authorization/StandardAuthorityProviderConfigurationContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/authorization/StandardAuthorityProviderConfigurationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/authorization/StandardAuthorityProviderConfigurationContext.java
deleted file mode 100644
index 45b84c8..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/authorization/StandardAuthorityProviderConfigurationContext.java
+++ /dev/null
@@ -1,51 +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.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- *
- */
-public class StandardAuthorityProviderConfigurationContext implements AuthorityProviderConfigurationContext {
-
-    private final String identifier;
-    private final Map<String, String> properties;
-
-    public StandardAuthorityProviderConfigurationContext(String identifier, Map<String, String> properties) {
-        this.identifier = identifier;
-        this.properties = Collections.unmodifiableMap(new HashMap<String, String>(properties));
-    }
-
-    @Override
-    public String getIdentifier() {
-        return identifier;
-    }
-
-    @Override
-    public Map<String, String> getProperties() {
-        return properties;
-    }
-
-    @Override
-    public String getProperty(String property) {
-        return properties.get(property);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/c4d06f20/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/authorization/StandardAuthorityProviderInitializationContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/authorization/StandardAuthorityProviderInitializationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/authorization/StandardAuthorityProviderInitializationContext.java
deleted file mode 100644
index e4b16c4..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/authorization/StandardAuthorityProviderInitializationContext.java
+++ /dev/null
@@ -1,42 +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;
-
-/**
- *
- */
-public class StandardAuthorityProviderInitializationContext implements AuthorityProviderInitializationContext {
-
-    private final String identifier;
-    private final AuthorityProviderLookup authorityProviderLookup;
-
-    public StandardAuthorityProviderInitializationContext(String identifier, AuthorityProviderLookup authorityProviderLookup) {
-        this.identifier = identifier;
-        this.authorityProviderLookup = authorityProviderLookup;
-    }
-
-    @Override
-    public String getIdentifier() {
-        return identifier;
-    }
-
-    @Override
-    public AuthorityProviderLookup getAuthorityProviderLookup() {
-        return authorityProviderLookup;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/c4d06f20/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/user/AccountStatus.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/user/AccountStatus.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/user/AccountStatus.java
deleted file mode 100644
index d7becf1..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/user/AccountStatus.java
+++ /dev/null
@@ -1,47 +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.user;
-
-/**
- * Represents the status of a user's account.
- */
-public enum AccountStatus {
-
-    ACTIVE,
-    PENDING,
-    DISABLED;
-
-    /**
-     * Returns the matching status or null if the specified status does not
-     * match any statuses.
-     *
-     * @param rawStatus string form of status
-     * @return account status object
-     */
-    public static AccountStatus valueOfStatus(String rawStatus) {
-        AccountStatus desiredStatus = null;
-
-        for (AccountStatus status : values()) {
-            if (status.toString().equals(rawStatus)) {
-                desiredStatus = status;
-                break;
-            }
-        }
-
-        return desiredStatus;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/c4d06f20/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/user/NiFiUser.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/user/NiFiUser.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/user/NiFiUser.java
index 231b133..3da7b3d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/user/NiFiUser.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/user/NiFiUser.java
@@ -17,123 +17,52 @@
 package org.apache.nifi.user;
 
 import java.io.Serializable;
-import java.util.Date;
-import java.util.EnumSet;
 import java.util.Objects;
-import java.util.Set;
-import org.apache.nifi.authorization.Authority;
-import org.apache.commons.lang3.StringUtils;
 
 /**
  * An NiFiUser.
  */
 public class NiFiUser implements Serializable {
 
-    public static final String ANONYMOUS_USER_IDENTITY = "anonymous";
+    public static final NiFiUser ANONYMOUS = new NiFiUser("anonymous");
 
-    private String id;
     private String identity;
     private String userName;
-    private String userGroup;
-    private String justification;
-
-    private Date creation;
-    private Date lastVerified;
-    private Date lastAccessed;
-
-    private AccountStatus status;
-    private EnumSet<Authority> authorities;
 
     private NiFiUser chain;
 
-    /* getters / setters */
-    public Date getCreation() {
-        return creation;
+    public NiFiUser(String identity) {
+        this(identity, null, null);
     }
 
-    public void setCreation(Date creation) {
-        this.creation = creation;
+    public NiFiUser(String identity, String userName) {
+        this(identity, userName, null);
     }
 
-    public String getIdentity() {
-        return identity;
+    public NiFiUser(String identity, NiFiUser chain) {
+        this(identity, null, chain);
     }
 
-    public void setIdentity(String identity) {
+    public NiFiUser(String identity, String userName, NiFiUser chain) {
         this.identity = identity;
-    }
-
-    public String getUserName() {
-        return userName;
-    }
-
-    public void setUserName(String userName) {
         this.userName = userName;
+        this.chain = chain;
     }
 
-    public String getUserGroup() {
-        return userGroup;
-    }
-
-    public void setUserGroup(String userGroup) {
-        this.userGroup = userGroup;
-    }
-
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    public String getJustification() {
-        return justification;
-    }
-
-    public void setJustification(String justification) {
-        this.justification = justification;
-    }
-
-    public AccountStatus getStatus() {
-        return status;
-    }
-
-    public void setStatus(AccountStatus status) {
-        this.status = status;
-    }
-
-    public Date getLastVerified() {
-        return lastVerified;
-    }
-
-    public void setLastVerified(Date lastVerified) {
-        this.lastVerified = lastVerified;
-    }
+    /* getters / setters */
 
-    public Date getLastAccessed() {
-        return lastAccessed;
+    public String getIdentity() {
+        return identity;
     }
 
-    public void setLastAccessed(Date lastAccessed) {
-        this.lastAccessed = lastAccessed;
+    public String getUserName() {
+        return userName;
     }
 
     public NiFiUser getChain() {
         return chain;
     }
 
-    public void setChain(NiFiUser chain) {
-        this.chain = chain;
-    }
-
-    public Set<Authority> getAuthorities() {
-        if (authorities == null) {
-            authorities = EnumSet.noneOf(Authority.class);
-        }
-        return authorities;
-    }
-
     @Override
     public boolean equals(Object obj) {
         if (obj == null) {
@@ -158,7 +87,7 @@ public class NiFiUser implements Serializable {
 
     @Override
     public String toString() {
-        return String.format("identity[%s], userName[%s], justification[%s], authorities[%s]", getIdentity(), getUserName(), getJustification(), StringUtils.join(getAuthorities(), ", "));
+        return String.format("identity[%s], userName[%s]", getIdentity(), getUserName(), ", ");
     }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/c4d06f20/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/resources/nifi-administration-context.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/resources/nifi-administration-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/resources/nifi-administration-context.xml
index 3a46314..bc3662c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/resources/nifi-administration-context.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/resources/nifi-administration-context.xml
@@ -18,41 +18,34 @@
        xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
        xsi:schemaLocation="http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans-3.1.xsd">
 
-    <!-- user authority provider -->
-    <bean id="authorityProvider" class="org.apache.nifi.authorization.AuthorityProviderFactoryBean" depends-on="clusterManager">
-        <property name="properties" ref="nifiProperties"/>
-    </bean>
-
     <!-- user/entity authorizer -->
-    <bean id="authorizer" class="org.apache.nifi.authorization.AuthorizerFactoryBean" depends-on="clusterManager">
+    <bean id="authorizer" class="org.apache.nifi.authorization.AuthorizerFactoryBean">
         <property name="properties" ref="nifiProperties"/>
     </bean>
 
-    <!-- initialize the user data source -->
-    <bean id="userDataSource" class="org.apache.nifi.admin.UserDataSourceFactoryBean" destroy-method="shutdown">
+    <!-- initialize the user key data source -->
+    <bean id="keyDataSource" class="org.apache.nifi.admin.KeyDataSourceFactoryBean" destroy-method="shutdown">
         <property name="properties" ref="nifiProperties"/>
     </bean>
 
-    <!-- initialize the data source -->
-    <bean id="auditDataSource" class="org.apache.nifi.admin.AuditDataSourceFactoryBean" destroy-method="shutdown" depends-on="userDataSource">
+    <!-- initialize the audit data source -->
+    <bean id="auditDataSource" class="org.apache.nifi.admin.AuditDataSourceFactoryBean" destroy-method="shutdown">
         <property name="properties" ref="nifiProperties"/>
     </bean>
-    
-    <!-- initialize the user transaction builder -->
-    <bean id="userTransactionBuilder" class="org.apache.nifi.admin.service.transaction.impl.StandardTransactionBuilder">
-        <property name="authorityProvider" ref="authorityProvider"/>
-        <property name="dataSource" ref="userDataSource"/>
+
+    <!-- initialize the user key transaction builder -->
+    <bean id="keyTransactionBuilder" class="org.apache.nifi.admin.service.transaction.impl.StandardTransactionBuilder">
+        <property name="dataSource" ref="keyDataSource"/>
     </bean>
-    
+
     <!-- initialize the audit transaction builder -->
     <bean id="auditTransactionBuilder" class="org.apache.nifi.admin.service.transaction.impl.StandardTransactionBuilder">
-        <property name="authorityProvider" ref="authorityProvider"/>
         <property name="dataSource" ref="auditDataSource"/>
     </bean>
-    
+
     <!-- administration service -->
-    <bean id="userService" class="org.apache.nifi.admin.service.impl.StandardUserService" init-method="seedUserAccounts">
-        <property name="transactionBuilder" ref="userTransactionBuilder"/>
+    <bean id="userService" class="org.apache.nifi.admin.service.impl.StandardUserService">
+        <property name="transactionBuilder" ref="keyTransactionBuilder"/>
         <property name="properties" ref="nifiProperties"/>
     </bean>
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/c4d06f20/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/xsd/authority-providers.xsd
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/xsd/authority-providers.xsd b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/xsd/authority-providers.xsd
deleted file mode 100644
index 1a5fe50..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/xsd/authority-providers.xsd
+++ /dev/null
@@ -1,49 +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="Provider">
-        <xs:sequence>
-            <xs:element name="identifier" type="AuthorityProviderNonEmptyStringType"/>
-            <xs:element name="class" type="AuthorityProviderNonEmptyStringType"/>
-            <xs:element name="property" type="AuthorityProviderProperty" minOccurs="0" maxOccurs="unbounded" />
-        </xs:sequence>
-    </xs:complexType>
-
-    <!-- Name/Value properties-->
-    <xs:complexType name="AuthorityProviderProperty">
-        <xs:simpleContent>
-            <xs:extension base="xs:string">
-                <xs:attribute name="name" type="AuthorityProviderNonEmptyStringType"></xs:attribute>
-            </xs:extension>
-        </xs:simpleContent>
-    </xs:complexType>
-
-    <xs:simpleType name="AuthorityProviderNonEmptyStringType">
-        <xs:restriction base="xs:string">
-            <xs:minLength value="1"/>
-        </xs:restriction>
-    </xs:simpleType>
-
-    <!-- users -->
-    <xs:element name="authorityProviders">
-        <xs:complexType>
-            <xs:sequence>
-                <xs:element name="provider" type="Provider" 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/c4d06f20/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/test/java/org/apache/nifi/admin/service/action/AuthorizeUserActionTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/test/java/org/apache/nifi/admin/service/action/AuthorizeUserActionTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/test/java/org/apache/nifi/admin/service/action/AuthorizeUserActionTest.java
deleted file mode 100644
index 8d3c15a..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/test/java/org/apache/nifi/admin/service/action/AuthorizeUserActionTest.java
+++ /dev/null
@@ -1,433 +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.admin.service.action;
-
-import java.util.Date;
-import java.util.EnumSet;
-import java.util.Set;
-import org.apache.nifi.admin.dao.AuthorityDAO;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-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.authorization.Authority;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.authorization.exception.AuthorityAccessException;
-import org.apache.nifi.authorization.exception.UnknownIdentityException;
-import org.apache.nifi.user.AccountStatus;
-import org.apache.nifi.user.NiFiUser;
-import org.apache.commons.lang3.StringUtils;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.ArgumentCaptor;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- *
- */
-public class AuthorizeUserActionTest {
-
-    private static final String USER_ID_6 = "6";
-    private static final String USER_ID_7 = "7";
-    private static final String USER_ID_8 = "8";
-    private static final String USER_ID_9 = "9";
-    private static final String USER_ID_10 = "10";
-    private static final String USER_ID_11 = "11";
-
-    private static final String USER_IDENTITY_1 = "authority access exception while searching for user";
-    private static final String USER_IDENTITY_2 = "unknown user";
-    private static final String USER_IDENTITY_3 = "user removed after checking existence";
-    private static final String USER_IDENTITY_4 = "access exception getting authorities";
-    private static final String USER_IDENTITY_5 = "error creating user account";
-    private static final String USER_IDENTITY_6 = "create user general sequence";
-    private static final String USER_IDENTITY_7 = "existing user requires verification";
-    private static final String USER_IDENTITY_8 = "existing user does not require verification";
-    private static final String USER_IDENTITY_9 = "existing pending user";
-    private static final String USER_IDENTITY_10 = "existing disabled user";
-    private static final String USER_IDENTITY_11 = "existing user is now unknown in the authority provider";
-
-    private DAOFactory daoFactory;
-    private UserDAO userDao;
-    private AuthorityDAO authorityDao;
-    private AuthorityProvider authorityProvider;
-
-    @Before
-    public void setup() throws Exception {
-        // mock the user dao
-        userDao = Mockito.mock(UserDAO.class);
-        Mockito.doAnswer(new Answer<NiFiUser>() {
-            @Override
-            public NiFiUser answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String id = (String) args[0];
-
-                NiFiUser user = null;
-                if (USER_ID_7.equals(id)) {
-                    user = new NiFiUser();
-                    user.setId(USER_ID_7);
-                    user.setIdentity(USER_IDENTITY_7);
-                    user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR));
-                } else if (USER_ID_8.equals(id)) {
-                    user = new NiFiUser();
-                    user.setId(USER_ID_8);
-                    user.setIdentity(USER_IDENTITY_8);
-                    user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR));
-                    user.setLastVerified(new Date());
-                } else if (USER_ID_11.equals(id)) {
-                    user = new NiFiUser();
-                    user.setId(USER_ID_11);
-                    user.setIdentity(USER_IDENTITY_11);
-                    user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR));
-                    user.setStatus(AccountStatus.ACTIVE);
-                }
-
-                return user;
-            }
-        }).when(userDao).findUserById(Mockito.anyString());
-        Mockito.doAnswer(new Answer<NiFiUser>() {
-            @Override
-            public NiFiUser answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String dn = (String) args[0];
-
-                NiFiUser user = null;
-                switch (dn) {
-                    case USER_IDENTITY_7:
-                        user = new NiFiUser();
-                        user.setId(USER_ID_7);
-                        user.setIdentity(USER_IDENTITY_7);
-                        user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR));
-                        break;
-                    case USER_IDENTITY_8:
-                        user = new NiFiUser();
-                        user.setId(USER_ID_8);
-                        user.setIdentity(USER_IDENTITY_8);
-                        user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR));
-                        user.setLastVerified(new Date());
-                        break;
-                    case USER_IDENTITY_9:
-                        user = new NiFiUser();
-                        user.setId(USER_ID_9);
-                        user.setIdentity(USER_IDENTITY_9);
-                        user.setStatus(AccountStatus.PENDING);
-                        break;
-                    case USER_IDENTITY_10:
-                        user = new NiFiUser();
-                        user.setId(USER_ID_10);
-                        user.setIdentity(USER_IDENTITY_10);
-                        user.setStatus(AccountStatus.DISABLED);
-                        break;
-                    case USER_IDENTITY_11:
-                        user = new NiFiUser();
-                        user.setId(USER_ID_11);
-                        user.setIdentity(USER_IDENTITY_11);
-                        user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR));
-                        user.setStatus(AccountStatus.ACTIVE);
-                        break;
-                }
-
-                return user;
-            }
-        }).when(userDao).findUserByDn(Mockito.anyString());
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                NiFiUser user = (NiFiUser) args[0];
-                switch (user.getIdentity()) {
-                    case USER_IDENTITY_5:
-                        throw new DataAccessException();
-                    case USER_IDENTITY_6:
-                        user.setId(USER_ID_6);
-                        break;
-                }
-
-                // do nothing
-                return null;
-            }
-        }).when(userDao).createUser(Mockito.any(NiFiUser.class));
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                NiFiUser user = (NiFiUser) args[0];
-
-                // do nothing
-                return null;
-            }
-        }).when(userDao).updateUser(Mockito.any(NiFiUser.class));
-
-        // mock the authority dao
-        authorityDao = Mockito.mock(AuthorityDAO.class);
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                Set<Authority> authorities = (Set<Authority>) args[0];
-                String id = (String) args[1];
-
-                // do nothing
-                return null;
-            }
-        }).when(authorityDao).createAuthorities(Mockito.anySetOf(Authority.class), Mockito.anyString());
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                Set<Authority> authorities = (Set<Authority>) args[0];
-                String id = (String) args[1];
-
-                // do nothing
-                return null;
-            }
-        }).when(authorityDao).deleteAuthorities(Mockito.anySetOf(Authority.class), Mockito.anyString());
-
-        // mock the dao factory
-        daoFactory = Mockito.mock(DAOFactory.class);
-        Mockito.when(daoFactory.getUserDAO()).thenReturn(userDao);
-        Mockito.when(daoFactory.getAuthorityDAO()).thenReturn(authorityDao);
-
-        // mock the authority provider
-        authorityProvider = Mockito.mock(AuthorityProvider.class);
-        Mockito.doAnswer(new Answer<Boolean>() {
-            @Override
-            public Boolean answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String dn = (String) args[0];
-                switch (dn) {
-                    case USER_IDENTITY_1:
-                        throw new AuthorityAccessException(StringUtils.EMPTY);
-                    case USER_IDENTITY_2:
-                        return false;
-                }
-
-                return true;
-            }
-        }).when(authorityProvider).doesDnExist(Mockito.anyString());
-        Mockito.doAnswer(new Answer<Set<Authority>>() {
-            @Override
-            public Set<Authority> answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String dn = (String) args[0];
-                Set<Authority> authorities = EnumSet.noneOf(Authority.class);
-                switch (dn) {
-                    case USER_IDENTITY_3:
-                        throw new UnknownIdentityException(StringUtils.EMPTY);
-                    case USER_IDENTITY_4:
-                        throw new AuthorityAccessException(StringUtils.EMPTY);
-                    case USER_IDENTITY_6:
-                        authorities.add(Authority.ROLE_MONITOR);
-                        break;
-                    case USER_IDENTITY_7:
-                        authorities.add(Authority.ROLE_DFM);
-                        break;
-                    case USER_IDENTITY_9:
-                        throw new UnknownIdentityException(StringUtils.EMPTY);
-                    case USER_IDENTITY_10:
-                        throw new UnknownIdentityException(StringUtils.EMPTY);
-                    case USER_IDENTITY_11:
-                        throw new UnknownIdentityException(StringUtils.EMPTY);
-                }
-
-                return authorities;
-            }
-        }).when(authorityProvider).getAuthorities(Mockito.anyString());
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String dn = (String) args[0];
-                Set<Authority> authorites = (Set<Authority>) args[1];
-
-                // do nothing
-                return null;
-            }
-        }).when(authorityProvider).setAuthorities(Mockito.anyString(), Mockito.anySet());
-    }
-
-    /**
-     * Tests AuthorityAccessException in doesDnExist.
-     *
-     * @throws Exception ex
-     */
-    @Test(expected = AdministrationException.class)
-    public void testAuthorityAccessExceptionInDoesDnExist() throws Exception {
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_IDENTITY_1, 0);
-        authorizeUser.execute(daoFactory, authorityProvider);
-    }
-
-    /**
-     * Test unknown user in the authority provider.
-     *
-     * @throws Exception ex
-     */
-    @Test(expected = AccountNotFoundException.class)
-    public void testUnknownUser() throws Exception {
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_IDENTITY_2, 0);
-        authorizeUser.execute(daoFactory, authorityProvider);
-    }
-
-    /**
-     * Test a user thats been removed after checking their existence.
-     *
-     * @throws Exception ex
-     */
-    @Test(expected = AccountNotFoundException.class)
-    public void testUserRemovedAfterCheckingExistence() throws Exception {
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_IDENTITY_3, 0);
-        authorizeUser.execute(daoFactory, authorityProvider);
-    }
-
-    /**
-     * Testing AuthorityAccessException when getting authorities.
-     *
-     * @throws Exception ex
-     */
-    @Test(expected = AdministrationException.class)
-    public void testAuthorityAccessException() throws Exception {
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_IDENTITY_4, 0);
-        authorizeUser.execute(daoFactory, authorityProvider);
-    }
-
-    /**
-     * Testing DataAccessException while creating user accounts.
-     *
-     * @throws Exception ex
-     */
-    @Test(expected = DataAccessException.class)
-    public void testErrorCreatingUserAccount() throws Exception {
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_IDENTITY_5, 0);
-        authorizeUser.execute(daoFactory, authorityProvider);
-    }
-
-    /**
-     * Tests the general case when a user account is created.
-     *
-     * @throws Exception ex
-     */
-    @Test
-    public void testAccountCreation() throws Exception {
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_IDENTITY_6, 0);
-        NiFiUser user = authorizeUser.execute(daoFactory, authorityProvider);
-
-        // verify the user
-        Assert.assertEquals(USER_IDENTITY_6, user.getIdentity());
-        Assert.assertEquals(1, user.getAuthorities().size());
-        Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_MONITOR));
-
-        // verify interaction with dao and provider
-        Mockito.verify(userDao, Mockito.times(1)).createUser(user);
-    }
-
-    /**
-     * Tests the general case when there is an existing user account that
-     * requires verification.
-     *
-     * @throws Exception ex
-     */
-    @Test
-    public void testExistingUserRequiresVerification() throws Exception {
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_IDENTITY_7, 0);
-        NiFiUser user = authorizeUser.execute(daoFactory, authorityProvider);
-
-        // verify the user
-        Assert.assertEquals(USER_IDENTITY_7, user.getIdentity());
-        Assert.assertEquals(1, user.getAuthorities().size());
-        Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_DFM));
-
-        // verify interaction with dao and provider
-        Mockito.verify(userDao, Mockito.times(1)).updateUser(user);
-        Mockito.verify(authorityDao, Mockito.times(1)).createAuthorities(EnumSet.of(Authority.ROLE_DFM), USER_ID_7);
-    }
-
-    /**
-     * Tests the general case when there is an existing user account that does
-     * not require verification.
-     *
-     * @throws Exception ex
-     */
-    @Test
-    public void testExistingUserNoVerification() throws Exception {
-        // disabling verification by passing in a large cache duration
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_IDENTITY_8, Integer.MAX_VALUE);
-        NiFiUser user = authorizeUser.execute(daoFactory, authorityProvider);
-
-        // verify the user
-        Assert.assertEquals(USER_IDENTITY_8, user.getIdentity());
-        Assert.assertEquals(1, user.getAuthorities().size());
-        Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_MONITOR));
-
-        // verify interaction with dao and provider
-        Mockito.verify(userDao, Mockito.times(1)).updateUser(user);
-        Mockito.verify(authorityDao, Mockito.never()).createAuthorities(Mockito.anySet(), Mockito.eq(USER_ID_8));
-        Mockito.verify(authorityDao, Mockito.never()).deleteAuthorities(Mockito.anySet(), Mockito.eq(USER_ID_8));
-    }
-
-    /**
-     * Tests existing users whose accounts are in a pending status.
-     *
-     * @throws Exception ex
-     */
-    @Test(expected = AccountPendingException.class)
-    public void testExistingPendingUser() throws Exception {
-        // disabling verification by passing in a large cache duration
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_IDENTITY_9, Integer.MAX_VALUE);
-        authorizeUser.execute(daoFactory, authorityProvider);
-    }
-
-    /**
-     * Tests existing users whose accounts are in a disabled status.
-     *
-     * @throws Exception ex
-     */
-    @Test(expected = AccountDisabledException.class)
-    public void testExistingDisabledUser() throws Exception {
-        // disabling verification by passing in a large cache duration
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_IDENTITY_10, Integer.MAX_VALUE);
-        authorizeUser.execute(daoFactory, authorityProvider);
-    }
-
-    /**
-     * Tests the general case where there is an active user that has been
-     * removed from the authority provider.
-     *
-     * @throws Exception ex
-     */
-    @Test
-    public void testExistingActiveUserNotFoundInProvider() throws Exception {
-        try {
-            AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_IDENTITY_11, 0);
-            authorizeUser.execute(daoFactory, authorityProvider);
-
-            Assert.fail();
-        } catch (AccountDisabledException ade) {
-            ArgumentCaptor<NiFiUser> user = ArgumentCaptor.forClass(NiFiUser.class);
-
-            // verify interaction with dao
-            Mockito.verify(userDao, Mockito.times(1)).updateUser(user.capture());
-
-            // verify user
-            Assert.assertEquals(AccountStatus.DISABLED, user.getValue().getStatus());
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/c4d06f20/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/test/java/org/apache/nifi/admin/service/action/CreateUserActionTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/test/java/org/apache/nifi/admin/service/action/CreateUserActionTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/test/java/org/apache/nifi/admin/service/action/CreateUserActionTest.java
deleted file mode 100644
index e372781..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/test/java/org/apache/nifi/admin/service/action/CreateUserActionTest.java
+++ /dev/null
@@ -1,144 +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.admin.service.action;
-
-import java.util.EnumSet;
-import java.util.Set;
-import org.apache.nifi.admin.dao.AuthorityDAO;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.authorization.Authority;
-import org.apache.nifi.user.NiFiUser;
-import org.apache.commons.lang3.StringUtils;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- * Test cases for creating a user.
- */
-public class CreateUserActionTest {
-
-    private final String USER_ID_2 = "2";
-    private final String USER_ID_3 = "3";
-
-    private final String USER_IDENTITY_1 = "data access exception when creating user";
-    private final String USER_IDENTITY_3 = "general create user case";
-
-    private DAOFactory daoFactory;
-    private UserDAO userDao;
-    private AuthorityDAO authorityDao;
-
-    @Before
-    public void setup() throws Exception {
-        // mock the user dao
-        userDao = Mockito.mock(UserDAO.class);
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                NiFiUser user = (NiFiUser) args[0];
-
-                if (USER_IDENTITY_1.equals(user.getIdentity())) {
-                    throw new DataAccessException();
-                } else if (USER_IDENTITY_3.equals(user.getIdentity())) {
-                    user.setId(USER_ID_3);
-                }
-
-                // do nothing
-                return null;
-            }
-        }).when(userDao).createUser(Mockito.any(NiFiUser.class));
-
-        // mock the authority dao
-        authorityDao = Mockito.mock(AuthorityDAO.class);
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                Set<Authority> authorities = (Set<Authority>) args[0];
-                String id = (String) args[1];
-
-                if (USER_ID_2.equals(id)) {
-                    throw new DataAccessException(StringUtils.EMPTY);
-                }
-
-                // do nothing
-                return null;
-            }
-        }).when(authorityDao).createAuthorities(Mockito.anySetOf(Authority.class), Mockito.anyString());
-
-        // mock the dao factory
-        daoFactory = Mockito.mock(DAOFactory.class);
-        Mockito.when(daoFactory.getUserDAO()).thenReturn(userDao);
-        Mockito.when(daoFactory.getAuthorityDAO()).thenReturn(authorityDao);
-    }
-
-    /**
-     * Tests DataAccessExceptions that occur while creating user accounts.
-     *
-     * @throws Exception ex
-     */
-    @Test(expected = DataAccessException.class)
-    public void testExceptionCreatingUser() throws Exception {
-        NiFiUser user = new NiFiUser();
-        user.setIdentity(USER_IDENTITY_1);
-
-        CreateUserAction createUser = new CreateUserAction(user);
-        createUser.execute(daoFactory, null);
-    }
-
-    /**
-     * Tests DataAccessExceptions that occur while create user authorities.
-     *
-     * @throws Exception ex
-     */
-    @Test(expected = DataAccessException.class)
-    public void testExceptionCreatingAuthoroties() throws Exception {
-        NiFiUser user = new NiFiUser();
-        user.setId(USER_ID_2);
-
-        CreateUserAction createUser = new CreateUserAction(user);
-        createUser.execute(daoFactory, null);
-    }
-
-    /**
-     * General case for creating a user.
-     *
-     * @throws Exception ex
-     */
-    @Test
-    public void testCreateUserAccount() throws Exception {
-        NiFiUser user = new NiFiUser();
-        user.setIdentity(USER_IDENTITY_3);
-        user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_DFM, Authority.ROLE_ADMIN));
-
-        CreateUserAction createUser = new CreateUserAction(user);
-        createUser.execute(daoFactory, null);
-
-        // verify the user
-        Assert.assertEquals(USER_ID_3, user.getId());
-
-        // verify interaction with dao
-        Mockito.verify(userDao, Mockito.times(1)).createUser(user);
-        Mockito.verify(authorityDao, Mockito.times(1)).createAuthorities(user.getAuthorities(), USER_ID_3);
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/c4d06f20/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/test/java/org/apache/nifi/admin/service/action/DisableUserActionTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/test/java/org/apache/nifi/admin/service/action/DisableUserActionTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/test/java/org/apache/nifi/admin/service/action/DisableUserActionTest.java
deleted file mode 100644
index b5f0a7f..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/test/java/org/apache/nifi/admin/service/action/DisableUserActionTest.java
+++ /dev/null
@@ -1,176 +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.admin.service.action;
-
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.admin.service.AccountNotFoundException;
-import org.apache.nifi.admin.service.AdministrationException;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.authorization.exception.AuthorityAccessException;
-import org.apache.nifi.user.AccountStatus;
-import org.apache.nifi.user.NiFiUser;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.admin.dao.KeyDAO;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-public class DisableUserActionTest {
-
-    private static final String USER_ID_1 = "1";
-    private static final String USER_ID_2 = "2";
-    private static final String USER_ID_3 = "3";
-    private static final String USER_ID_4 = "4";
-
-    private static final String USER_IDENTITY_3 = "authority access exception";
-    private static final String USER_IDENTITY_4 = "general disable user case";
-
-    private DAOFactory daoFactory;
-    private UserDAO userDao;
-    private KeyDAO keyDao;
-    private AuthorityProvider authorityProvider;
-
-    @Before
-    public void setup() throws Exception {
-        // mock the user dao
-        userDao = Mockito.mock(UserDAO.class);
-        Mockito.doAnswer(new Answer<NiFiUser>() {
-            @Override
-            public NiFiUser answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String id = (String) args[0];
-
-                NiFiUser user = null;
-                if (USER_ID_1.equals(id)) {
-                    // leave user uninitialized
-                } else if (USER_ID_2.equals(id)) {
-                    user = new NiFiUser();
-                    user.setId(id);
-                } else if (USER_ID_3.equals(id)) {
-                    user = new NiFiUser();
-                    user.setId(id);
-                    user.setIdentity(USER_IDENTITY_3);
-                } else if (USER_ID_4.equals(id)) {
-                    user = new NiFiUser();
-                    user.setId(id);
-                    user.setIdentity(USER_IDENTITY_4);
-                    user.setStatus(AccountStatus.ACTIVE);
-                }
-                return user;
-            }
-        }).when(userDao).findUserById(Mockito.anyString());
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                NiFiUser user = (NiFiUser) args[0];
-
-                if (USER_ID_2.equals(user.getId())) {
-                    throw new DataAccessException(StringUtils.EMPTY);
-                }
-
-                // do nothing
-                return null;
-            }
-        }).when(userDao).updateUser(Mockito.any(NiFiUser.class));
-
-        // mock the dao factory
-        keyDao = Mockito.mock(KeyDAO.class);
-        Mockito.doNothing().when(keyDao).deleteKeys(Matchers.anyString());
-
-        // mock the dao factory
-        daoFactory = Mockito.mock(DAOFactory.class);
-        Mockito.when(daoFactory.getUserDAO()).thenReturn(userDao);
-        Mockito.when(daoFactory.getKeyDAO()).thenReturn(keyDao);
-
-        // mock the authority provider
-        authorityProvider = Mockito.mock(AuthorityProvider.class);
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String dn = (String) args[0];
-
-                if (USER_IDENTITY_3.equals(dn)) {
-                    throw new AuthorityAccessException(StringUtils.EMPTY);
-                }
-
-                // do nothing
-                return null;
-            }
-        }).when(authorityProvider).revokeUser(Mockito.anyString());
-    }
-
-    /**
-     * Tests the case when the user account is unknown.
-     *
-     * @throws Exception ex
-     */
-    @Test(expected = AccountNotFoundException.class)
-    public void testUnknownUserAccount() throws Exception {
-        DisableUserAction disableUser = new DisableUserAction(USER_ID_1);
-        disableUser.execute(daoFactory, authorityProvider);
-    }
-
-    /**
-     * Tests the case when a DataAccessException is thrown by the userDao.
-     *
-     * @throws Exception ex
-     */
-    @Test(expected = DataAccessException.class)
-    public void testDataAccessExceptionInUserDao() throws Exception {
-        DisableUserAction disableUser = new DisableUserAction(USER_ID_2);
-        disableUser.execute(daoFactory, authorityProvider);
-    }
-
-    /**
-     * Tests the case when a AuthorityAccessException is thrown by the provider.
-     *
-     * @throws Exception ex
-     */
-    @Test(expected = AdministrationException.class)
-    public void testAuthorityAccessExceptionInProvider() throws Exception {
-        DisableUserAction disableUser = new DisableUserAction(USER_ID_3);
-        disableUser.execute(daoFactory, authorityProvider);
-    }
-
-    /**
-     * Tests the general case when the user is disabled.
-     *
-     * @throws Exception ex
-     */
-    @Test
-    public void testDisableUser() throws Exception {
-        DisableUserAction disableUser = new DisableUserAction(USER_ID_4);
-        NiFiUser user = disableUser.execute(daoFactory, authorityProvider);
-
-        // verify the user
-        Assert.assertEquals(USER_ID_4, user.getId());
-        Assert.assertEquals(USER_IDENTITY_4, user.getIdentity());
-        Assert.assertEquals(AccountStatus.DISABLED, user.getStatus());
-
-        // verify the interaction with the dao and provider
-        Mockito.verify(userDao, Mockito.times(1)).updateUser(user);
-        Mockito.verify(authorityProvider, Mockito.times(1)).revokeUser(USER_IDENTITY_4);
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/c4d06f20/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/test/java/org/apache/nifi/admin/service/action/InvalidateUserAccountActionTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/test/java/org/apache/nifi/admin/service/action/InvalidateUserAccountActionTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/test/java/org/apache/nifi/admin/service/action/InvalidateUserAccountActionTest.java
deleted file mode 100644
index cffd280..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/test/java/org/apache/nifi/admin/service/action/InvalidateUserAccountActionTest.java
+++ /dev/null
@@ -1,126 +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.admin.service.action;
-
-import java.util.Date;
-import org.junit.Assert;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.admin.service.AccountNotFoundException;
-import org.apache.nifi.user.NiFiUser;
-import org.apache.commons.lang3.StringUtils;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.ArgumentCaptor;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- * Test case for InvalidateUserAccountAction.
- */
-public class InvalidateUserAccountActionTest {
-
-    private static final String USER_ID_1 = "1";
-    private static final String USER_ID_2 = "2";
-    private static final String USER_ID_3 = "3";
-
-    private DAOFactory daoFactory;
-    private UserDAO userDao;
-
-    @Before
-    public void setup() throws Exception {
-        // mock the user dao
-        userDao = Mockito.mock(UserDAO.class);
-        Mockito.doAnswer(new Answer<NiFiUser>() {
-            @Override
-            public NiFiUser answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String id = (String) args[0];
-
-                NiFiUser user = null;
-                if (USER_ID_1.equals(id)) {
-                    // leave uninitialized
-                } else if (USER_ID_2.equals(id)) {
-                    user = new NiFiUser();
-                    user.setId(USER_ID_2);
-                } else if (USER_ID_3.equals(id)) {
-                    user = new NiFiUser();
-                    user.setId(USER_ID_3);
-                    user.setLastVerified(new Date());
-                }
-                return user;
-            }
-        }).when(userDao).findUserById(Mockito.anyString());
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                NiFiUser user = (NiFiUser) args[0];
-
-                if (USER_ID_2.equals(user.getId())) {
-                    throw new DataAccessException(StringUtils.EMPTY);
-                }
-
-                // do nothing
-                return null;
-            }
-        }).when(userDao).updateUser(Mockito.any(NiFiUser.class));
-
-        // mock the dao factory
-        daoFactory = Mockito.mock(DAOFactory.class);
-        Mockito.when(daoFactory.getUserDAO()).thenReturn(userDao);
-    }
-
-    @Test(expected = AccountNotFoundException.class)
-    public void testAccountNotFoundException() throws Exception {
-        InvalidateUserAccountAction invalidateUserAccount = new InvalidateUserAccountAction(USER_ID_1);
-        invalidateUserAccount.execute(daoFactory, null);
-    }
-
-    /**
-     * Tests when a data access exception occurs when updating the user record.
-     *
-     * @throws Exception ex
-     */
-    @Test(expected = DataAccessException.class)
-    public void testDataAccessException() throws Exception {
-        InvalidateUserAccountAction invalidateUserAccount = new InvalidateUserAccountAction(USER_ID_2);
-        invalidateUserAccount.execute(daoFactory, null);
-    }
-
-    /**
-     * Tests the general case of invalidating a user.
-     *
-     * @throws Exception ex
-     */
-    @Test
-    public void testInvalidateUser() throws Exception {
-        InvalidateUserAccountAction invalidateUserAccount = new InvalidateUserAccountAction(USER_ID_3);
-        invalidateUserAccount.execute(daoFactory, null);
-
-        // verify the interaction with the dao
-        ArgumentCaptor<NiFiUser> userCaptor = ArgumentCaptor.forClass(NiFiUser.class);
-        Mockito.verify(userDao, Mockito.times(1)).updateUser(userCaptor.capture());
-
-        // verify the user
-        NiFiUser user = userCaptor.getValue();
-        Assert.assertEquals(USER_ID_3, user.getId());
-        Assert.assertNull(user.getLastVerified());
-    }
-}