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:03:56 UTC

[04/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-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/user/NiFiUserUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/user/NiFiUserUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/user/NiFiUserUtils.java
index 341663e..255b3d5 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/user/NiFiUserUtils.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/user/NiFiUserUtils.java
@@ -77,27 +77,6 @@ public final class NiFiUserUtils {
         return user;
     }
 
-    /**
-     * Returns the NewAccountRequest or null if this is not a new account request.
-     *
-     * @return new account request
-     */
-    public static NewAccountRequest getNewAccountRequest() {
-        NewAccountRequest newAccountRequest = null;
-
-        // obtain the principal in the current authentication
-        final SecurityContext context = SecurityContextHolder.getContext();
-        final Authentication authentication = context.getAuthentication();
-        if (authentication != null) {
-            Object principal = authentication.getPrincipal();
-            if (principal instanceof NewAccountRequest) {
-                newAccountRequest = (NewAccountRequest) principal;
-            }
-        }
-
-        return newAccountRequest;
-    }
-
     public static String getNiFiUserName() {
         // get the nifi user to extract the username
         NiFiUser user = NiFiUserUtils.getNiFiUser();

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/X509AuthenticationFilter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/X509AuthenticationFilter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/X509AuthenticationFilter.java
index 019a53c..ab6ceec 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/X509AuthenticationFilter.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/X509AuthenticationFilter.java
@@ -16,18 +16,15 @@
  */
 package org.apache.nifi.web.security.x509;
 
-import java.security.cert.X509Certificate;
-import java.util.List;
-import javax.servlet.http.HttpServletRequest;
-import org.apache.nifi.authentication.AuthenticationResponse;
-import org.apache.nifi.web.security.InvalidAuthenticationException;
 import org.apache.nifi.web.security.NiFiAuthenticationFilter;
 import org.apache.nifi.web.security.ProxiedEntitiesUtils;
-import org.apache.nifi.web.security.token.NewAccountAuthorizationRequestToken;
-import org.apache.nifi.web.security.token.NiFiAuthorizationRequestToken;
-import org.apache.nifi.web.security.user.NewAccountRequest;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.web.authentication.preauth.x509.X509PrincipalExtractor;
+
+import javax.servlet.http.HttpServletRequest;
+import java.security.cert.X509Certificate;
 
 /**
  * Custom X509 filter that will inspect the HTTP headers for a proxied user before extracting the user details from the client certificate.
@@ -37,10 +34,10 @@ public class X509AuthenticationFilter extends NiFiAuthenticationFilter {
     private static final Logger logger = LoggerFactory.getLogger(X509AuthenticationFilter.class);
 
     private X509CertificateExtractor certificateExtractor;
-    private X509IdentityProvider certificateIdentityProvider;
+    private X509PrincipalExtractor principalExtractor;
 
     @Override
-    public NiFiAuthorizationRequestToken attemptAuthentication(final HttpServletRequest request) {
+    public Authentication attemptAuthentication(final HttpServletRequest request) {
         // only suppport x509 login when running securely
         if (!request.isSecure()) {
             return null;
@@ -52,20 +49,7 @@ public class X509AuthenticationFilter extends NiFiAuthenticationFilter {
             return null;
         }
 
-        // attempt to authenticate if certificates were found
-        final AuthenticationResponse authenticationResponse;
-        try {
-            authenticationResponse = certificateIdentityProvider.authenticate(certificates);
-        } catch (final IllegalArgumentException iae) {
-            throw new InvalidAuthenticationException(iae.getMessage(), iae);
-        }
-
-        final List<String> proxyChain = ProxiedEntitiesUtils.buildProxiedEntitiesChain(request, authenticationResponse.getIdentity());
-        if (isNewAccountRequest(request)) {
-            return new NewAccountAuthorizationRequestToken(new NewAccountRequest(proxyChain, getJustification(request)));
-        } else {
-            return new NiFiAuthorizationRequestToken(proxyChain);
-        }
+        return new X509AuthenticationRequestToken(request.getHeader(ProxiedEntitiesUtils.PROXY_ENTITIES_CHAIN), principalExtractor, certificates);
     }
 
     /* setters */
@@ -73,8 +57,8 @@ public class X509AuthenticationFilter extends NiFiAuthenticationFilter {
         this.certificateExtractor = certificateExtractor;
     }
 
-    public void setCertificateIdentityProvider(X509IdentityProvider certificateIdentityProvider) {
-        this.certificateIdentityProvider = certificateIdentityProvider;
+    public void setPrincipalExtractor(X509PrincipalExtractor principalExtractor) {
+        this.principalExtractor = principalExtractor;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/X509AuthenticationProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/X509AuthenticationProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/X509AuthenticationProvider.java
new file mode 100644
index 0000000..2593f92
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/X509AuthenticationProvider.java
@@ -0,0 +1,78 @@
+/*
+ * 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.web.security.x509;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.authentication.AuthenticationResponse;
+import org.apache.nifi.user.NiFiUser;
+import org.apache.nifi.web.security.InvalidAuthenticationException;
+import org.apache.nifi.web.security.ProxiedEntitiesUtils;
+import org.apache.nifi.web.security.token.NiFiAuthenticationToken;
+import org.apache.nifi.web.security.user.NiFiUserDetails;
+import org.springframework.security.authentication.AuthenticationProvider;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.core.AuthenticationException;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.ListIterator;
+
+/**
+ *
+ */
+public class X509AuthenticationProvider implements AuthenticationProvider {
+
+    private X509IdentityProvider certificateIdentityProvider;
+
+    public X509AuthenticationProvider(X509IdentityProvider certificateIdentityProvider) {
+        this.certificateIdentityProvider = certificateIdentityProvider;
+    }
+
+    @Override
+    public Authentication authenticate(Authentication authentication) throws AuthenticationException {
+        final X509AuthenticationRequestToken request = (X509AuthenticationRequestToken) authentication;
+
+        // attempt to authenticate if certificates were found
+        final AuthenticationResponse authenticationResponse;
+        try {
+            authenticationResponse = certificateIdentityProvider.authenticate(request.getCertificates());
+        } catch (final IllegalArgumentException iae) {
+            throw new InvalidAuthenticationException(iae.getMessage(), iae);
+        }
+
+        if (StringUtils.isBlank(request.getProxiedEntitiesChain())) {
+            return new NiFiAuthenticationToken(new NiFiUserDetails(new NiFiUser(authenticationResponse.getIdentity(), authenticationResponse.getUsername(), null)));
+        } else {
+            // build the entire proxy chain if applicable - <end-user><proxy1><proxy2>
+            final List<String> proxyChain = new ArrayList<>(ProxiedEntitiesUtils.tokenizeProxiedEntitiesChain(request.getProxiedEntitiesChain()));
+            proxyChain.add(authenticationResponse.getIdentity());
+
+            // add the chain as appropriate to each proxy
+            NiFiUser proxy = null;
+            for (final ListIterator<String> chainIter = proxyChain.listIterator(proxyChain.size()); chainIter.hasPrevious();) {
+                proxy = new NiFiUser(chainIter.previous(), proxy);
+            }
+
+            return new NiFiAuthenticationToken(new NiFiUserDetails(proxy));
+        }
+    }
+
+    @Override
+    public boolean supports(Class<?> authentication) {
+        return X509AuthenticationRequestToken.class.isAssignableFrom(authentication);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/X509AuthenticationRequestToken.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/X509AuthenticationRequestToken.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/X509AuthenticationRequestToken.java
new file mode 100644
index 0000000..cec72fe
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/X509AuthenticationRequestToken.java
@@ -0,0 +1,75 @@
+/*
+ * 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.web.security.x509;
+
+import org.apache.commons.lang3.StringUtils;
+import org.springframework.security.authentication.AbstractAuthenticationToken;
+import org.springframework.security.web.authentication.preauth.x509.X509PrincipalExtractor;
+
+import java.security.cert.X509Certificate;
+
+/**
+ * This is an authentication request with a given JWT token.
+ */
+public class X509AuthenticationRequestToken extends AbstractAuthenticationToken {
+
+    private final String proxiedEntitiesChain;
+    private final X509PrincipalExtractor principalExtractor;
+    private final X509Certificate[] certificates;
+
+    /**
+     * Creates a representation of the jwt authentication request for a user.
+     *
+     * @param proxiedEntitiesChain   The http servlet request
+     * @param certificates  The certificate chain
+     */
+    public X509AuthenticationRequestToken(final String proxiedEntitiesChain, final X509PrincipalExtractor principalExtractor, final X509Certificate[] certificates) {
+        super(null);
+        setAuthenticated(false);
+        this.proxiedEntitiesChain = proxiedEntitiesChain;
+        this.principalExtractor = principalExtractor;
+        this.certificates = certificates;
+    }
+
+    @Override
+    public Object getCredentials() {
+        return null;
+    }
+
+    @Override
+    public Object getPrincipal() {
+        if (StringUtils.isBlank(proxiedEntitiesChain)) {
+            return principalExtractor.extractPrincipal(certificates[0]);
+        } else {
+            return String.format("%s<%s>", proxiedEntitiesChain, principalExtractor.extractPrincipal(certificates[0]));
+        }
+    }
+
+    public String getProxiedEntitiesChain() {
+        return proxiedEntitiesChain;
+    }
+
+    public X509Certificate[] getCertificates() {
+        return certificates;
+    }
+
+    @Override
+    public String toString() {
+        return getName();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/ocsp/OcspCertificateValidator.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/ocsp/OcspCertificateValidator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/ocsp/OcspCertificateValidator.java
index b0762b5..108926c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/ocsp/OcspCertificateValidator.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/ocsp/OcspCertificateValidator.java
@@ -121,9 +121,8 @@ public class OcspCertificateValidator {
                     trustedCAs.put(ocspCertificate.getSubjectX500Principal().getName(), ocspCertificate);
                 }
 
-                // determine how long to cache the ocsp responses for
-                final String rawCacheDurationDuration = properties.getUserCredentialCacheDuration();
-                final long cacheDurationMillis = FormatUtils.getTimeDuration(rawCacheDurationDuration, TimeUnit.MILLISECONDS);
+                // TODO - determine how long to cache the ocsp responses for
+                final long cacheDurationMillis = FormatUtils.getTimeDuration("12 hours", TimeUnit.MILLISECONDS);
 
                 // build the ocsp cache
                 ocspCache = CacheBuilder.newBuilder().expireAfterWrite(cacheDurationMillis, TimeUnit.MILLISECONDS).build(new CacheLoader<OcspRequest, OcspStatus>() {

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/resources/nifi-web-security-context.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/resources/nifi-web-security-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/resources/nifi-web-security-context.xml
index 4e24bad..12d8594 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/resources/nifi-web-security-context.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/resources/nifi-web-security-context.xml
@@ -39,22 +39,31 @@
         <property name="certificateValidator" ref="certificateValidator"/>
     </bean>
 
-    <!-- user details service -->
-    <bean id="userDetailsService" class="org.apache.nifi.web.security.authorization.NiFiAuthorizationService">
-        <property name="userService" ref="userService"/>
-        <property name="properties" ref="nifiProperties"/>
+    <!-- otp authentication provider -->
+    <bean id="x509AuthenticationProvider" class="org.apache.nifi.web.security.x509.X509AuthenticationProvider">
+        <constructor-arg ref="certificateIdentityProvider"/>
     </bean>
 
     <!-- jwt service -->
     <bean id="jwtService" class="org.apache.nifi.web.security.jwt.JwtService">
-        <constructor-arg ref="userService"/>
+        <constructor-arg ref="keyService"/>
+    </bean>
+
+    <!-- jwt authentication provider -->
+    <bean id="jwtAuthenticationProvider" class="org.apache.nifi.web.security.jwt.JwtAuthenticationProvider">
+        <constructor-arg ref="jwtService"/>
     </bean>
 
     <!-- otp service -->
     <bean id="otpService" class="org.apache.nifi.web.security.otp.OtpService"/>
 
+    <!-- otp authentication provider -->
+    <bean id="otpAuthenticationProvider" class="org.apache.nifi.web.security.otp.OtpAuthenticationProvider">
+        <constructor-arg ref="otpService"/>
+    </bean>
+
     <!-- Kerberos service -->
-    <bean id="kerberosService" class="org.apache.nifi.web.security.kerberos.KerberosServiceFactoryBean">
+    <bean id="kerberosService" class="org.apache.nifi.web.security.spring.KerberosServiceFactoryBean">
         <property name="properties" ref="nifiProperties"/>
     </bean>
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/authorization/NiFiAuthorizationServiceTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/authorization/NiFiAuthorizationServiceTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/authorization/NiFiAuthorizationServiceTest.java
deleted file mode 100644
index 23b49b7..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/authorization/NiFiAuthorizationServiceTest.java
+++ /dev/null
@@ -1,249 +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.web.security.authorization;
-
-import java.util.Arrays;
-import org.apache.nifi.admin.service.AccountDisabledException;
-import org.apache.nifi.admin.service.AccountNotFoundException;
-import org.apache.nifi.admin.service.AccountPendingException;
-import org.apache.nifi.admin.service.AdministrationException;
-import org.apache.nifi.admin.service.UserService;
-import org.apache.nifi.authorization.Authority;
-import org.apache.nifi.user.NiFiUser;
-import org.apache.nifi.util.NiFiProperties;
-import org.apache.nifi.web.security.UntrustedProxyException;
-import org.apache.nifi.web.security.token.NiFiAuthorizationRequestToken;
-import org.apache.nifi.web.security.user.NiFiUserDetails;
-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;
-import org.springframework.security.authentication.AccountStatusException;
-import org.springframework.security.authentication.AuthenticationServiceException;
-import org.springframework.security.core.userdetails.UsernameNotFoundException;
-
-/**
- * Test case for NiFiAuthorizationService.
- */
-public class NiFiAuthorizationServiceTest {
-
-    private static final String USER = "user";
-    private static final String PROXY = "proxy";
-    private static final String PROXY_PROXY = "proxy-proxy";
-    private static final String USER_NOT_FOUND = "user-not-found";
-    private static final String USER_DISABLED = "user-disabled";
-    private static final String USER_PENDING = "user-pending";
-    private static final String USER_ADMIN_EXCEPTION = "user-admin-exception";
-    private static final String PROXY_NOT_FOUND = "proxy-not-found";
-
-    private NiFiAuthorizationService authorizationService;
-    private UserService userService;
-
-    @Before
-    public void setup() throws Exception {
-        // mock the web security properties
-        final NiFiProperties properties = Mockito.mock(NiFiProperties.class);
-        Mockito.when(properties.getSupportNewAccountRequests()).thenReturn(Boolean.TRUE);
-
-        userService = Mockito.mock(UserService.class);
-        Mockito.doReturn(null).when(userService).createPendingUserAccount(Mockito.anyString(), Mockito.anyString());
-        Mockito.doAnswer(new Answer() {
-            @Override
-            public Object answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String identity = (String) args[0];
-
-                if (null != identity) {
-                    switch (identity) {
-                        case USER_NOT_FOUND:
-                        case PROXY_NOT_FOUND:
-                            throw new AccountNotFoundException("");
-                        case USER_DISABLED:
-                            throw new AccountDisabledException("");
-                        case USER_PENDING:
-                            throw new AccountPendingException("");
-                        case USER_ADMIN_EXCEPTION:
-                            throw new AdministrationException();
-                        case USER:
-                            final NiFiUser monitor = new NiFiUser();
-                            monitor.setIdentity(identity);
-                            monitor.getAuthorities().add(Authority.ROLE_MONITOR);
-                            return monitor;
-                        case PROXY:
-                        case PROXY_PROXY:
-                            final NiFiUser proxy = new NiFiUser();
-                            proxy.setIdentity(identity);
-                            proxy.getAuthorities().add(Authority.ROLE_PROXY);
-                            return proxy;
-                    }
-                }
-
-                return null;
-            }
-        }).when(userService).checkAuthorization(Mockito.anyString());
-
-        // create the authorization service
-        authorizationService = new NiFiAuthorizationService();
-        authorizationService.setProperties(properties);
-        authorizationService.setUserService(userService);
-    }
-
-    private NiFiAuthorizationRequestToken createRequestAuthentication(final String... identities) {
-        return new NiFiAuthorizationRequestToken(Arrays.asList(identities));
-    }
-
-    /**
-     * Ensures the authorization service correctly handles users invalid identity chain.
-     *
-     * @throws Exception ex
-     */
-    @Test(expected = UntrustedProxyException.class)
-    public void testInvalidDnChain() throws Exception {
-        authorizationService.loadUserDetails(createRequestAuthentication());
-    }
-
-    /**
-     * Ensures the authorization service correctly handles account not found.
-     *
-     * @throws Exception ex
-     */
-    @Test(expected = UsernameNotFoundException.class)
-    public void testAccountNotFound() throws Exception {
-        authorizationService.loadUserDetails(createRequestAuthentication(USER_NOT_FOUND));
-    }
-
-    /**
-     * Ensures the authorization service correctly handles account disabled.
-     *
-     * @throws Exception ex
-     */
-    @Test(expected = AccountStatusException.class)
-    public void testAccountDisabled() throws Exception {
-        authorizationService.loadUserDetails(createRequestAuthentication(USER_DISABLED));
-    }
-
-    /**
-     * Ensures the authorization service correctly handles account pending.
-     *
-     * @throws Exception ex
-     */
-    @Test(expected = AccountStatusException.class)
-    public void testAccountPending() throws Exception {
-        authorizationService.loadUserDetails(createRequestAuthentication(USER_PENDING));
-    }
-
-    /**
-     * Ensures the authorization service correctly handles account administration exception.
-     *
-     * @throws Exception ex
-     */
-    @Test(expected = AuthenticationServiceException.class)
-    public void testAccountAdminException() throws Exception {
-        authorizationService.loadUserDetails(createRequestAuthentication(USER_ADMIN_EXCEPTION));
-    }
-
-    /**
-     * Tests the case when there is no proxy.
-     *
-     * @throws Exception ex
-     */
-    @Test
-    public void testNoProxy() throws Exception {
-        final NiFiUserDetails details = (NiFiUserDetails) authorizationService.loadUserDetails(createRequestAuthentication(USER));
-        final NiFiUser user = details.getNiFiUser();
-
-        Assert.assertEquals(USER, user.getIdentity());
-        Assert.assertNull(user.getChain());
-    }
-
-    /**
-     * Tests the case when the proxy does not have ROLE_PROXY.
-     *
-     * @throws Exception ex
-     */
-    @Test(expected = UntrustedProxyException.class)
-    public void testInvalidProxy() throws Exception {
-        authorizationService.loadUserDetails(createRequestAuthentication(USER, USER));
-    }
-
-    /**
-     * Ensures the authorization service correctly handles proxy not found by attempting to create an account request for the proxy.
-     *
-     * @throws Exception ex
-     */
-    @Test(expected = UntrustedProxyException.class)
-    public void testProxyNotFound() throws Exception {
-        try {
-            authorizationService.loadUserDetails(createRequestAuthentication(USER, PROXY_NOT_FOUND));
-        } finally {
-            Mockito.verify(userService).createPendingUserAccount(Mockito.eq(PROXY_NOT_FOUND), Mockito.anyString());
-        }
-    }
-
-    /**
-     * Tests the case when there is a proxy.
-     *
-     * @throws Exception ex
-     */
-    @Test
-    public void testProxy() throws Exception {
-        final NiFiUserDetails details = (NiFiUserDetails) authorizationService.loadUserDetails(createRequestAuthentication(USER, PROXY));
-        final NiFiUser user = details.getNiFiUser();
-
-        // verify the user
-        Assert.assertEquals(USER, user.getIdentity());
-        Assert.assertNotNull(user.getChain());
-
-        // get the proxy
-        final NiFiUser proxy = user.getChain();
-
-        // verify the proxy
-        Assert.assertEquals(PROXY, proxy.getIdentity());
-        Assert.assertNull(proxy.getChain());
-    }
-
-    /**
-     * Tests the case when there is are multiple proxies.
-     *
-     * @throws Exception ex
-     */
-    @Test
-    public void testProxyProxy() throws Exception {
-        final NiFiUserDetails details = (NiFiUserDetails) authorizationService.loadUserDetails(createRequestAuthentication(USER, PROXY, PROXY_PROXY));
-        final NiFiUser user = details.getNiFiUser();
-
-        // verify the user
-        Assert.assertEquals(USER, user.getIdentity());
-        Assert.assertNotNull(user.getChain());
-
-        // get the proxy
-        NiFiUser proxy = user.getChain();
-
-        // verify the proxy
-        Assert.assertEquals(PROXY, proxy.getIdentity());
-        Assert.assertNotNull(proxy.getChain());
-
-        // get the proxies proxy
-        proxy = proxy.getChain();
-
-        // verify the proxies proxy
-        Assert.assertEquals(PROXY_PROXY, proxy.getIdentity());
-        Assert.assertNull(proxy.getChain());
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/jwt/JwtServiceTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/jwt/JwtServiceTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/jwt/JwtServiceTest.java
index 658f3e6..59c66ef 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/jwt/JwtServiceTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/jwt/JwtServiceTest.java
@@ -20,7 +20,7 @@ import io.jsonwebtoken.JwtException;
 import org.apache.commons.codec.CharEncoding;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.nifi.admin.service.AdministrationException;
-import org.apache.nifi.admin.service.UserService;
+import org.apache.nifi.admin.service.KeyService;
 import org.apache.nifi.key.Key;
 import org.apache.nifi.web.security.token.LoginAuthenticationToken;
 import org.codehaus.jettison.json.JSONObject;
@@ -131,7 +131,7 @@ public class JwtServiceTest {
 
     private static final String HMAC_SECRET = "test_hmac_shared_secret";
 
-    private UserService mockUserService;
+    private KeyService mockKeyService;
 
     // Class under test
     private JwtService jwtService;
@@ -177,10 +177,10 @@ public class JwtServiceTest {
         key.setIdentity(DEFAULT_IDENTITY);
         key.setKey(HMAC_SECRET);
 
-        mockUserService = Mockito.mock(UserService.class);
-        when(mockUserService.getKey(anyInt())).thenReturn(key);
-        when(mockUserService.getOrCreateKey(anyString())).thenReturn(key);
-        jwtService = new JwtService(mockUserService);
+        mockKeyService = Mockito.mock(KeyService.class);
+        when(mockKeyService.getKey(anyInt())).thenReturn(key);
+        when(mockKeyService.getOrCreateKey(anyString())).thenReturn(key);
+        jwtService = new JwtService(mockKeyService);
     }
 
     @After
@@ -431,7 +431,7 @@ public class JwtServiceTest {
         logger.debug("Generating token for " + loginAuthenticationToken);
 
         // Set up the bad key service
-        UserService missingKeyService = Mockito.mock(UserService.class);
+        KeyService missingKeyService = Mockito.mock(KeyService.class);
         when(missingKeyService.getOrCreateKey(anyString())).thenThrow(new AdministrationException("Could not find a "
                 + "key for that user"));
         jwtService = new JwtService(missingKeyService);

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/otp/OtpAuthenticationFilterTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/otp/OtpAuthenticationFilterTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/otp/OtpAuthenticationFilterTest.java
index ad6f722..791ca54 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/otp/OtpAuthenticationFilterTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/otp/OtpAuthenticationFilterTest.java
@@ -16,24 +16,17 @@
  */
 package org.apache.nifi.web.security.otp;
 
-import org.apache.nifi.web.security.token.NiFiAuthorizationRequestToken;
 import org.junit.Before;
 import org.junit.Test;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
 
 import javax.servlet.http.HttpServletRequest;
-import java.util.List;
 import java.util.UUID;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.doAnswer;
+import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
 public class OtpAuthenticationFilterTest {
@@ -44,41 +37,11 @@ public class OtpAuthenticationFilterTest {
     private final static String DOWNLOAD_AUTHENTICATED_USER = "download-token-authenticated-user";
     private final static String DOWNLOAD_TOKEN = "download-token";
 
-    private OtpService otpService;
     private OtpAuthenticationFilter otpAuthenticationFilter;
 
     @Before
     public void setUp() throws Exception {
-        otpService = mock(OtpService.class);
-        doAnswer(new Answer<String>() {
-            @Override
-            public String answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String downloadToken = (String) args[0];
-
-                if (DOWNLOAD_TOKEN.equals(downloadToken)) {
-                    return DOWNLOAD_AUTHENTICATED_USER;
-                }
-
-                throw new OtpAuthenticationException("Invalid token");
-            }
-        }).when(otpService).getAuthenticationFromDownloadToken(anyString());
-        doAnswer(new Answer<String>() {
-            @Override
-            public String answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String uiExtensionToken = (String) args[0];
-
-                if (UI_EXTENSION_TOKEN.equals(uiExtensionToken)) {
-                    return UI_EXTENSION_AUTHENTICATED_USER;
-                }
-
-                throw new OtpAuthenticationException("Invalid token");
-            }
-        }).when(otpService).getAuthenticationFromUiExtensionToken(anyString());
-
         otpAuthenticationFilter = new OtpAuthenticationFilter();
-        otpAuthenticationFilter.setOtpService(otpService);
     }
 
     @Test
@@ -114,13 +77,9 @@ public class OtpAuthenticationFilterTest {
         when(request.getParameter(OtpAuthenticationFilter.ACCESS_TOKEN)).thenReturn(UI_EXTENSION_TOKEN);
         when(request.getContextPath()).thenReturn("/nifi-update-attribute-ui");
 
-        final NiFiAuthorizationRequestToken result = otpAuthenticationFilter.attemptAuthentication(request);
-        final List<String> chain = result.getChain();
-        assertEquals(1, chain.size());
-        assertEquals(UI_EXTENSION_AUTHENTICATED_USER, chain.get(0));
-
-        verify(otpService, times(1)).getAuthenticationFromUiExtensionToken(UI_EXTENSION_TOKEN);
-        verify(otpService, never()).getAuthenticationFromDownloadToken(anyString());
+        final OtpAuthenticationRequestToken result = (OtpAuthenticationRequestToken) otpAuthenticationFilter.attemptAuthentication(request);
+        assertEquals(UI_EXTENSION_TOKEN, result.getToken());
+        assertFalse(result.isDownloadToken());
     }
 
     @Test
@@ -131,13 +90,9 @@ public class OtpAuthenticationFilterTest {
         when(request.getContextPath()).thenReturn("/nifi-api");
         when(request.getPathInfo()).thenReturn("/controller/provenance/events/0/content/input");
 
-        final NiFiAuthorizationRequestToken result = otpAuthenticationFilter.attemptAuthentication(request);
-        final List<String> chain = result.getChain();
-        assertEquals(1, chain.size());
-        assertEquals(DOWNLOAD_AUTHENTICATED_USER, chain.get(0));
-
-        verify(otpService, never()).getAuthenticationFromUiExtensionToken(anyString());
-        verify(otpService, times(1)).getAuthenticationFromDownloadToken(DOWNLOAD_TOKEN);
+        final OtpAuthenticationRequestToken result = (OtpAuthenticationRequestToken) otpAuthenticationFilter.attemptAuthentication(request);
+        assertEquals(DOWNLOAD_TOKEN, result.getToken());
+        assertTrue(result.isDownloadToken());
     }
 
     @Test
@@ -148,13 +103,9 @@ public class OtpAuthenticationFilterTest {
         when(request.getContextPath()).thenReturn("/nifi-api");
         when(request.getPathInfo()).thenReturn("/controller/provenance/events/0/content/output");
 
-        final NiFiAuthorizationRequestToken result = otpAuthenticationFilter.attemptAuthentication(request);
-        final List<String> chain = result.getChain();
-        assertEquals(1, chain.size());
-        assertEquals(DOWNLOAD_AUTHENTICATED_USER, chain.get(0));
-
-        verify(otpService, never()).getAuthenticationFromUiExtensionToken(anyString());
-        verify(otpService, times(1)).getAuthenticationFromDownloadToken(DOWNLOAD_TOKEN);
+        final OtpAuthenticationRequestToken result = (OtpAuthenticationRequestToken) otpAuthenticationFilter.attemptAuthentication(request);
+        assertEquals(DOWNLOAD_TOKEN, result.getToken());
+        assertTrue(result.isDownloadToken());
     }
 
     @Test
@@ -167,13 +118,9 @@ public class OtpAuthenticationFilterTest {
         when(request.getContextPath()).thenReturn("/nifi-api");
         when(request.getPathInfo()).thenReturn(String.format("/controller/process-groups/root/connections/%s/flowfiles/%s/content", uuid, uuid));
 
-        final NiFiAuthorizationRequestToken result = otpAuthenticationFilter.attemptAuthentication(request);
-        final List<String> chain = result.getChain();
-        assertEquals(1, chain.size());
-        assertEquals(DOWNLOAD_AUTHENTICATED_USER, chain.get(0));
-
-        verify(otpService, never()).getAuthenticationFromUiExtensionToken(anyString());
-        verify(otpService, times(1)).getAuthenticationFromDownloadToken(DOWNLOAD_TOKEN);
+        final OtpAuthenticationRequestToken result = (OtpAuthenticationRequestToken) otpAuthenticationFilter.attemptAuthentication(request);
+        assertEquals(DOWNLOAD_TOKEN, result.getToken());
+        assertTrue(result.isDownloadToken());
     }
 
     @Test
@@ -186,13 +133,9 @@ public class OtpAuthenticationFilterTest {
         when(request.getContextPath()).thenReturn("/nifi-api");
         when(request.getPathInfo()).thenReturn(String.format("/controller/templates/%s", uuid));
 
-        final NiFiAuthorizationRequestToken result = otpAuthenticationFilter.attemptAuthentication(request);
-        final List<String> chain = result.getChain();
-        assertEquals(1, chain.size());
-        assertEquals(DOWNLOAD_AUTHENTICATED_USER, chain.get(0));
-
-        verify(otpService, never()).getAuthenticationFromUiExtensionToken(anyString());
-        verify(otpService, times(1)).getAuthenticationFromDownloadToken(DOWNLOAD_TOKEN);
+        final OtpAuthenticationRequestToken result = (OtpAuthenticationRequestToken) otpAuthenticationFilter.attemptAuthentication(request);
+        assertEquals(DOWNLOAD_TOKEN, result.getToken());
+        assertTrue(result.isDownloadToken());
     }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/otp/OtpAuthenticationProviderTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/otp/OtpAuthenticationProviderTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/otp/OtpAuthenticationProviderTest.java
new file mode 100644
index 0000000..a95c1a0
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/otp/OtpAuthenticationProviderTest.java
@@ -0,0 +1,102 @@
+/*
+ * 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.web.security.otp;
+
+import org.apache.nifi.web.security.token.NiFiAuthenticationToken;
+import org.apache.nifi.web.security.user.NiFiUserDetails;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+public class OtpAuthenticationProviderTest {
+
+    private final static String UI_EXTENSION_AUTHENTICATED_USER = "ui-extension-token-authenticated-user";
+    private final static String UI_EXTENSION_TOKEN = "ui-extension-token";
+
+    private final static String DOWNLOAD_AUTHENTICATED_USER = "download-token-authenticated-user";
+    private final static String DOWNLOAD_TOKEN = "download-token";
+
+    private OtpService otpService;
+    private OtpAuthenticationProvider otpAuthenticationProvider;
+
+    @Before
+    public void setUp() throws Exception {
+        otpService = mock(OtpService.class);
+        doAnswer(new Answer<String>() {
+            @Override
+            public String answer(InvocationOnMock invocation) throws Throwable {
+                Object[] args = invocation.getArguments();
+                String downloadToken = (String) args[0];
+
+                if (DOWNLOAD_TOKEN.equals(downloadToken)) {
+                    return DOWNLOAD_AUTHENTICATED_USER;
+                }
+
+                throw new OtpAuthenticationException("Invalid token");
+            }
+        }).when(otpService).getAuthenticationFromDownloadToken(anyString());
+        doAnswer(new Answer<String>() {
+            @Override
+            public String answer(InvocationOnMock invocation) throws Throwable {
+                Object[] args = invocation.getArguments();
+                String uiExtensionToken = (String) args[0];
+
+                if (UI_EXTENSION_TOKEN.equals(uiExtensionToken)) {
+                    return UI_EXTENSION_AUTHENTICATED_USER;
+                }
+
+                throw new OtpAuthenticationException("Invalid token");
+            }
+        }).when(otpService).getAuthenticationFromUiExtensionToken(anyString());
+
+        otpAuthenticationProvider = new OtpAuthenticationProvider(otpService);
+    }
+
+    @Test
+    public void testUiExtensionPath() throws Exception {
+        final OtpAuthenticationRequestToken request = new OtpAuthenticationRequestToken(UI_EXTENSION_TOKEN, false);
+
+        final NiFiAuthenticationToken result = (NiFiAuthenticationToken) otpAuthenticationProvider.authenticate(request);
+        final NiFiUserDetails details = (NiFiUserDetails) result.getPrincipal();
+        assertEquals(UI_EXTENSION_AUTHENTICATED_USER, details.getUsername());
+
+        verify(otpService, times(1)).getAuthenticationFromUiExtensionToken(UI_EXTENSION_TOKEN);
+        verify(otpService, never()).getAuthenticationFromDownloadToken(anyString());
+    }
+
+    @Test
+    public void testDownload() throws Exception {
+        final OtpAuthenticationRequestToken request = new OtpAuthenticationRequestToken(DOWNLOAD_TOKEN, true);
+
+        final NiFiAuthenticationToken result = (NiFiAuthenticationToken) otpAuthenticationProvider.authenticate(request);
+        final NiFiUserDetails details = (NiFiUserDetails) result.getPrincipal();
+        assertEquals(DOWNLOAD_AUTHENTICATED_USER, details.getUsername());
+
+        verify(otpService, never()).getAuthenticationFromUiExtensionToken(anyString());
+        verify(otpService, times(1)).getAuthenticationFromDownloadToken(DOWNLOAD_TOKEN);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
index 3d9a7d7..c798191 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
@@ -32,7 +32,6 @@
         <counters.filter>counters.properties</counters.filter>
         <cluster.filter>cluster.properties</cluster.filter>
         <templates.filter>templates.properties</templates.filter>
-        <users.filter>users.properties</users.filter>
         <bulletin.board.filter>bulletin-board.properties</bulletin.board.filter>
         <login.filter>login.properties</login.filter>
         <provenance.filter>provenance.properties</provenance.filter>
@@ -55,7 +54,6 @@
             <filter>src/main/resources/filters/${counters.filter}</filter>
             <filter>src/main/resources/filters/${cluster.filter}</filter>
             <filter>src/main/resources/filters/${templates.filter}</filter>
-            <filter>src/main/resources/filters/${users.filter}</filter>
             <filter>src/main/resources/filters/${bulletin.board.filter}</filter>
             <filter>src/main/resources/filters/${login.filter}</filter>
             <filter>src/main/resources/filters/${provenance.filter}</filter>
@@ -93,7 +91,6 @@
                                 **/counters.jsp,
                                 **/cluster.jsp,
                                 **/templates.jsp,
-                                **/users.jsp,
                                 **/bulletin-board.jsp,
                                 **/login.jsp
                             </excludes>
@@ -195,14 +192,6 @@
                             <directory>src/main/webapp/WEB-INF/pages</directory>
                             <targetPath>WEB-INF/pages</targetPath>
                             <includes>
-                                <include>users.jsp</include>
-                            </includes>
-                            <filtering>true</filtering>
-                        </resource>
-                        <resource>
-                            <directory>src/main/webapp/WEB-INF/pages</directory>
-                            <targetPath>WEB-INF/pages</targetPath>
-                            <includes>
                                 <include>cluster.jsp</include>
                             </includes>
                             <filtering>true</filtering>
@@ -241,7 +230,6 @@
                 <counters.filter>counters-min.properties</counters.filter>
                 <cluster.filter>cluster-min.properties</cluster.filter>
                 <templates.filter>templates-min.properties</templates.filter>
-                <users.filter>users-min.properties</users.filter>
                 <bulletin.board.filter>bulletin-board-min.properties</bulletin.board.filter>
                 <login.filter>login-min.properties</login.filter>
                 <provenance.filter>provenance-min.properties</provenance.filter>
@@ -297,8 +285,6 @@
                                                 <include>${staging.dir}/js/nf/canvas/nf-remote-process-group-ports.js</include>
                                                 <include>${staging.dir}/js/nf/canvas/nf-port-configuration.js</include>
                                                 <include>${staging.dir}/js/nf/canvas/nf-port-details.js</include>
-                                                <include>${staging.dir}/js/nf/canvas/nf-secure-port-configuration.js</include>
-                                                <include>${staging.dir}/js/nf/canvas/nf-secure-port-details.js</include>
                                                 <include>${staging.dir}/js/nf/canvas/nf-label-configuration.js</include>
                                                 <include>${staging.dir}/js/nf/canvas/nf-connection-configuration.js</include>
                                                 <include>${staging.dir}/js/nf/nf-connection-details.js</include>
@@ -424,20 +410,6 @@
                                         </aggregation>
                                         <aggregation>
                                             <insertNewLine>true</insertNewLine>
-                                            <output>${project.build.directory}/${project.build.finalName}/js/nf/users/nf-users-all.js</output>
-                                            <includes>
-                                                <include>${staging.dir}/js/nf/nf-client.js</include>
-                                                <include>${staging.dir}/js/nf/nf-common.js</include>
-                                                <include>${staging.dir}/js/nf/nf-universal-capture.js</include>
-                                                <include>${staging.dir}/js/nf/nf-dialog.js</include>
-                                                <include>${staging.dir}/js/nf/nf-storage.js</include>
-                                                <include>${staging.dir}/js/nf/nf-ajax-setup.js</include>
-                                                <include>${staging.dir}/js/nf/users/nf-users.js</include>
-                                                <include>${staging.dir}/js/nf/users/nf-users-table.js</include>
-                                            </includes>
-                                        </aggregation>
-                                        <aggregation>
-                                            <insertNewLine>true</insertNewLine>
                                             <output>${project.build.directory}/${project.build.finalName}/js/nf/bulletin-board/nf-bulletin-board-all.js</output>
                                             <includes>
                                                 <include>${staging.dir}/js/nf/nf-client.js</include>
@@ -561,16 +533,6 @@
                                         </aggregation>
                                         <aggregation>
                                             <insertNewLine>true</insertNewLine>
-                                            <output>${project.build.directory}/${project.build.finalName}/css/nf-users-all.css</output>
-                                            <includes>
-                                                <include>${staging.dir}/css/main.css</include>
-                                                <include>${staging.dir}/css/banner.css</include>
-                                                <include>${staging.dir}/css/dialog.css</include>
-                                                <include>${staging.dir}/css/users.css</include>
-                                            </includes>
-                                        </aggregation>
-                                        <aggregation>
-                                            <insertNewLine>true</insertNewLine>
                                             <output>${project.build.directory}/${project.build.finalName}/css/nf-bulletin-board-all.css</output>
                                             <includes>
                                                 <include>${staging.dir}/css/main.css</include>
@@ -624,8 +586,6 @@
                                 css/nf-cluster-all.css.gz,
                                 css/nf-templates-all.css,
                                 css/nf-templates-all.css.gz,
-                                css/nf-users-all.css,
-                                css/nf-users-all.css.gz,
                                 css/nf-bulletin-board-all.css,
                                 css/nf-bulletin-board-all.css.gz,
                                 css/nf-login-all.css,
@@ -665,8 +625,6 @@
                                 js/nf/cluster/nf-cluster-all.js.gz,
                                 js/nf/templates/nf-templates-all.js,
                                 js/nf/templates/nf-templates-all.js.gz,
-                                js/nf/users/nf-users-all.js,
-                                js/nf/users/nf-users-all.js.gz,
                                 js/nf/bulletin-board/nf-bulletin-board-all.js,
                                 js/nf/bulletin-board/nf-bulletin-board-all.js.gz,
                                 js/nf/login/nf-login-all.js,

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/canvas.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/canvas.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/canvas.properties
index ab42f86..67a02f4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/canvas.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/canvas.properties
@@ -38,8 +38,6 @@ nf.canvas.script.tags=<script type="text/javascript" src="js/nf/nf-namespace.js?
 <script type="text/javascript" src="js/nf/canvas/nf-remote-process-group-ports.js?${project.version}"></script>\n\
 <script type="text/javascript" src="js/nf/canvas/nf-port-configuration.js?${project.version}"></script>\n\
 <script type="text/javascript" src="js/nf/canvas/nf-port-details.js?${project.version}"></script>\n\
-<script type="text/javascript" src="js/nf/canvas/nf-secure-port-configuration.js?${project.version}"></script>\n\
-<script type="text/javascript" src="js/nf/canvas/nf-secure-port-details.js?${project.version}"></script>\n\
 <script type="text/javascript" src="js/nf/canvas/nf-label-configuration.js?${project.version}"></script>\n\
 <script type="text/javascript" src="js/nf/canvas/nf-connection-configuration.js?${project.version}"></script>\n\
 <script type="text/javascript" src="js/nf/nf-connection-details.js?${project.version}"></script>\n\

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/users-min.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/users-min.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/users-min.properties
deleted file mode 100644
index 9375334..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/users-min.properties
+++ /dev/null
@@ -1,18 +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.
-
-nf.users.script.tags=<script type="text/javascript" src="js/nf/users/nf-users-all.js?${project.version}"></script>
-nf.users.style.tags=<link rel="stylesheet" href="css/nf-users-all.css?${project.version}" type="text/css" />\n\
-<link rel="stylesheet" href="css/message-pane.css?${project.version}" type="text/css" />
\ 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-web/nifi-web-ui/src/main/resources/filters/users.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/users.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/users.properties
deleted file mode 100644
index 0b37363..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/users.properties
+++ /dev/null
@@ -1,29 +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.
-
-nf.users.script.tags=<script type="text/javascript" src="js/nf/nf-namespace.js?${project.version}"></script>\n\
-<script type="text/javascript" src="js/nf/nf-common.js?${project.version}"></script>\n\
-<script type="text/javascript" src="js/nf/nf-universal-capture.js?${project.version}"></script>\n\
-<script type="text/javascript" src="js/nf/nf-dialog.js?${project.version}"></script>\n\
-<script type="text/javascript" src="js/nf/nf-storage.js?${project.version}"></script>\n\
-<script type="text/javascript" src="js/nf/nf-ajax-setup.js?${project.version}"></script>\n\
-<script type="text/javascript" src="js/nf/users/nf-users.js?${project.version}"></script>\n\
-<script type="text/javascript" src="js/nf/users/nf-users-table.js?${project.version}"></script>
-nf.users.style.tags=<link rel="stylesheet" href="css/reset.css?${project.version}" type="text/css" />\n\
-<link rel="stylesheet" href="css/main.css?${project.version}" type="text/css" />\n\
-<link rel="stylesheet" href="css/banner.css?${project.version}" type="text/css" />\n\
-<link rel="stylesheet" href="css/dialog.css?${project.version}" type="text/css" />\n\
-<link rel="stylesheet" href="css/message-pane.css?${project.version}" type="text/css" />\n\
-<link rel="stylesheet" href="css/users.css?${project.version}" type="text/css" />
\ 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-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/canvas.jsp
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/canvas.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/canvas.jsp
index c6fe35d..fbab590 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/canvas.jsp
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/canvas.jsp
@@ -124,8 +124,6 @@
         <jsp:include page="/WEB-INF/partials/canvas/remote-port-configuration.jsp"/>
         <jsp:include page="/WEB-INF/partials/canvas/port-configuration.jsp"/>
         <jsp:include page="/WEB-INF/partials/canvas/port-details.jsp"/>
-        <jsp:include page="/WEB-INF/partials/canvas/secure-port-configuration.jsp"/>
-        <jsp:include page="/WEB-INF/partials/canvas/secure-port-details.jsp"/>
         <jsp:include page="/WEB-INF/partials/canvas/label-configuration.jsp"/>
         <jsp:include page="/WEB-INF/partials/canvas/connection-configuration.jsp"/>
         <jsp:include page="/WEB-INF/partials/canvas/drop-request-status-dialog.jsp"/>

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/users.jsp
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/users.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/users.jsp
deleted file mode 100644
index b3e0968..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/users.jsp
+++ /dev/null
@@ -1,72 +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.
---%>
-<%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %>
-<!DOCTYPE html>
-<html>
-    <head>
-        <title>NiFi Users</title>
-        <meta http-equiv="Content-Type" content="text/html; charset=UTF-8"/>
-        <link rel="shortcut icon" href="images/nifi16.ico"/>
-        <link rel="stylesheet" href="css/reset.css" type="text/css" />
-        ${nf.users.style.tags}
-        <link rel="stylesheet" href="js/jquery/tabbs/jquery.tabbs.css?${project.version}" type="text/css" />
-        <link rel="stylesheet" href="js/jquery/combo/jquery.combo.css?${project.version}" type="text/css" />
-        <link rel="stylesheet" href="js/jquery/modal/jquery.modal.css?${project.version}" type="text/css" />
-        <link rel="stylesheet" href="js/jquery/qtip2/jquery.qtip.min.css?" type="text/css" />
-        <link rel="stylesheet" href="js/jquery/ui-smoothness/jquery-ui-1.10.4.min.css" type="text/css" />
-        <link rel="stylesheet" href="js/jquery/slickgrid/css/slick.grid.css" type="text/css" />
-        <link rel="stylesheet" href="js/jquery/slickgrid/css/slick-default-theme.css" type="text/css" />
-        <script type="text/javascript" src="js/jquery/jquery-2.1.1.min.js"></script>
-        <script type="text/javascript" src="js/jquery/jquery.base64.js"></script>
-        <script type="text/javascript" src="js/jquery/jquery.center.js"></script>
-        <script type="text/javascript" src="js/jquery/tabbs/jquery.tabbs.js?${project.version}"></script>
-        <script type="text/javascript" src="js/jquery/combo/jquery.combo.js?${project.version}"></script>
-        <script type="text/javascript" src="js/jquery/modal/jquery.modal.js?${project.version}"></script>
-        <script type="text/javascript" src="js/jquery/jquery.ellipsis.js"></script>
-        <script type="text/javascript" src="js/jquery/jquery.each.js"></script>
-        <script type="text/javascript" src="js/jquery/ui-smoothness/jquery-ui-1.10.4.min.js"></script>
-        <script type="text/javascript" src="js/jquery/qtip2/jquery.qtip.min.js"></script>
-        <script type="text/javascript" src="js/jquery/jquery.event.drag-2.2.min.js"></script>
-        <script type="text/javascript" src="js/jquery/slickgrid/plugins/slick.cellrangeselector.js"></script>
-        <script type="text/javascript" src="js/jquery/slickgrid/plugins/slick.cellselectionmodel.js"></script>
-        <script type="text/javascript" src="js/jquery/slickgrid/plugins/slick.rowselectionmodel.js"></script>
-        <script type="text/javascript" src="js/jquery/slickgrid/plugins/slick.autotooltips.js"></script>
-        <script type="text/javascript" src="js/jquery/slickgrid/slick.formatters.js"></script>
-        <script type="text/javascript" src="js/jquery/slickgrid/slick.editors.js"></script>
-        <script type="text/javascript" src="js/jquery/slickgrid/slick.dataview.js"></script>
-        <script type="text/javascript" src="js/jquery/slickgrid/slick.core.js"></script>
-        <script type="text/javascript" src="js/jquery/slickgrid/slick.grid.js"></script>
-        <script type="text/javascript" src="js/nf/nf-namespace.js?${project.version}"></script>
-        ${nf.users.script.tags}
-    </head>
-    <body>
-        <jsp:include page="/WEB-INF/partials/message-pane.jsp"/>
-        <jsp:include page="/WEB-INF/partials/banners-utility.jsp"/>
-        <jsp:include page="/WEB-INF/partials/yes-no-dialog.jsp"/>
-        <jsp:include page="/WEB-INF/partials/ok-dialog.jsp"/>
-        <jsp:include page="/WEB-INF/partials/users/users-content.jsp"/>
-        <jsp:include page="/WEB-INF/partials/users/user-details-dialog.jsp"/>
-        <jsp:include page="/WEB-INF/partials/users/user-roles-dialog.jsp"/>
-        <jsp:include page="/WEB-INF/partials/users/group-roles-dialog.jsp"/>
-        <jsp:include page="/WEB-INF/partials/users/user-delete-dialog.jsp"/>
-        <jsp:include page="/WEB-INF/partials/users/user-revoke-dialog.jsp"/>
-        <jsp:include page="/WEB-INF/partials/users/group-revoke-dialog.jsp"/>
-        <jsp:include page="/WEB-INF/partials/users/user-group-dialog.jsp"/>
-        <div id="faded-background"></div>
-        <div id="glass-pane"></div>
-    </body>
-</html>

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/canvas-header.jsp
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/canvas-header.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/canvas-header.jsp
index 7a1d22d..ed93e43 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/canvas-header.jsp
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/canvas-header.jsp
@@ -33,7 +33,6 @@
                 <div id="provenance-link" class="utility-button" title="Data Provenance"></div>
                 <div id="flow-settings-link" class="utility-button" title="Controller Settings"></div>
                 <div id="templates-link" class="utility-button" title="Templates"></div>
-                <div id="users-link" class="utility-button" title="Users"><div id="has-pending-accounts" class="hidden"></div></div>
                 <div id="cluster-link" class="utility-button" title="Cluster"></div>
                 <div id="bulletin-board-link" class="utility-button" title="Bulletin Board"></div>
             </div>

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/secure-port-configuration.jsp
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/secure-port-configuration.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/secure-port-configuration.jsp
deleted file mode 100644
index bd87018..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/secure-port-configuration.jsp
+++ /dev/null
@@ -1,82 +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.
---%>
-<%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %>
-<div id="secure-port-configuration">
-    <div class="dialog-content">
-        <span id="secure-port-type" class="hidden"></span>
-        <div id="secure-port-configuration-tabs"></div>
-        <div id="secure-port-configuration-tabs-content">
-            <div id="secure-port-settings-tab-content" class="configuration-tab">
-                <div class="secure-port-setting">
-                    <div class="setting-name">Port name</div>
-                    <div class="setting-field">
-                        <input type="text" id="secure-port-name"/>
-                        <div class="port-enabled-container">
-                            <div id="secure-port-enabled" class="port-enabled nf-checkbox checkbox-unchecked"></div>
-                            <span> Enabled</span>
-                        </div>
-                        <div class="clear"></div>
-                    </div>
-                </div>
-                <div class="secure-port-setting">
-                    <div class="setting-name">
-                        Id
-                    </div>
-                    <div class="setting-field">
-                        <span id="secure-port-id"></span>
-                    </div>
-                </div>
-                <div id="secure-port-concurrent-task-container" class="secure-port-setting">
-                    <div class="setting-name">
-                        Concurrent tasks
-                        <img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="The number of tasks that should be concurrently scheduled for this port."/>
-                    </div>
-                    <div class="setting-field">
-                        <input type="text" id="secure-port-concurrent-tasks" class="secure-port-field"></input>
-                    </div>
-                </div>
-                <div class="secure-port-setting">
-                    <div class="setting-name">Comments</div>
-                    <div class="setting-field">
-                        <textarea cols="30" rows="4" id="secure-port-comments" class="secure-port-field"></textarea>
-                    </div>
-                </div>
-            </div>
-            <div id="secure-port-access-control-tab-content" class="configuration-tab">
-                <div class="secure-port-setting">
-                    <div class="setting-name">Search Users</div>
-                    <div class="setting-field">
-                        <input type="text" id="secure-port-access-control" class="secure-port-field"/>
-                    </div>
-                </div>
-                <div class="secure-port-setting">
-                    <div class="setting-name">Allowed Users</div>
-                    <div class="setting-field allowed-container">
-                        <ul id="allowed-users" class="allowed"></ul>
-                    </div>
-                </div>
-                <div class="secure-port-setting">
-                    <div class="setting-name">Allowed Groups</div>
-                    <div class="setting-field allowed-container">
-                        <ul id="allowed-groups" class="allowed"></ul>
-                    </div>
-                </div>
-            </div>
-        </div>
-    </div>
-</div>
-<div id="search-users-results"></div>
\ 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-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/secure-port-details.jsp
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/secure-port-details.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/secure-port-details.jsp
deleted file mode 100644
index 7b62450..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/secure-port-details.jsp
+++ /dev/null
@@ -1,67 +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.
---%>
-<%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %>
-<div id="secure-port-details">
-    <div class="dialog-content">
-        <div id="secure-port-details-tabs"></div>
-        <div id="secure-port-details-tabs-content">
-            <div id="read-only-secure-port-settings-tab-content" class="configuration-tab">
-                <div class="secure-port-setting">
-                    <div class="setting-name">Port name</div>
-                    <div class="setting-field">
-                        <div id="read-only-secure-port-name"></div>
-                    </div>
-                </div>
-                <div class="secure-port-setting">
-                    <div class="setting-name">Id</div>
-                    <div class="setting-field">
-                        <span id="read-only-secure-port-id"></span>
-                    </div>
-                </div>
-                <div id="secure-port-concurrent-task-container" class="secure-port-setting">
-                    <div class="setting-name">
-                        Concurrent tasks
-                        <img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="The number of tasks that should be concurrently scheduled for this port."/>
-                    </div>
-                    <div class="setting-field">
-                        <div id="read-only-secure-port-concurrent-tasks"></div>
-                    </div>
-                </div>
-                <div class="secure-port-setting">
-                    <div class="setting-name">Comments</div>
-                    <div class="setting-field">
-                        <div id="read-only-secure-port-comments"></div>
-                    </div>
-                </div>
-            </div>
-            <div id="read-only-secure-port-access-control-tab-content" class="configuration-tab">
-                <div class="secure-port-setting">
-                    <div class="setting-name">Allowed Users</div>
-                    <div class="setting-field allowed-container">
-                        <ul id="read-only-allowed-users" class="allowed"></ul>
-                    </div>
-                </div>
-                <div class="secure-port-setting">
-                    <div class="setting-name">Allowed Groups</div>
-                    <div class="setting-field allowed-container">
-                        <ul id="read-only-allowed-groups" class="allowed"></ul>
-                    </div>
-                </div>
-            </div>
-        </div>
-    </div>
-</div>
\ 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-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/users/group-revoke-dialog.jsp
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/users/group-revoke-dialog.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/users/group-revoke-dialog.jsp
deleted file mode 100644
index 436e28d..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/users/group-revoke-dialog.jsp
+++ /dev/null
@@ -1,22 +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.
---%>
-<%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %>
-<div id="group-revoke-dialog">
-    <div class="dialog-content">
-        Are you sure you want to revoke access for all users in '<span id="group-name-revoke-dialog"></span>'?
-    </div>
-</div>

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/users/group-roles-dialog.jsp
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/users/group-roles-dialog.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/users/group-roles-dialog.jsp
deleted file mode 100644
index f0db958..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/users/group-roles-dialog.jsp
+++ /dev/null
@@ -1,52 +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.
---%>
-<%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %>
-<div id="group-roles-dialog">
-    <div class="dialog-content">
-        <div class="setting">
-            <div class="setting-name">Group</div>
-            <div class="setting-field">
-                <span id="group-name-roles-dialog"></span>
-            </div>
-            <div class="clear"></div>
-        </div>
-        <div class="setting">
-            <div class="setting-name">Roles</div>
-            <div class="group-roles-container">
-                <div class="role-container">
-                    <div id="group-role-admin-checkbox" class="role-checkbox nf-checkbox checkbox-unchecked"></div><div class="role-name">Administrator</div>
-                </div>
-                <div class="role-container">
-                    <div id="group-role-dfm-checkbox" class="role-checkbox nf-checkbox checkbox-unchecked"></div><div class="role-name">Data Flow Manager</div>
-                </div>
-                <div class="role-container">
-                    <div id="group-role-monitor-checkbox" class="role-checkbox nf-checkbox checkbox-unchecked"></div><div class="role-name">Read Only</div>
-                </div>
-                <div class="role-container" style="margin-top: 5px;">
-                    <div id="group-role-provenance-checkbox" class="role-checkbox nf-checkbox checkbox-unchecked"></div><div class="role-name">Provenance</div>
-                </div>
-                <div class="role-container" style="margin-top: 5px;">
-                    <div id="group-role-nifi-checkbox" class="role-checkbox nf-checkbox checkbox-unchecked"></div><div class="role-name">NiFi</div>
-                </div>
-                <div class="role-container">
-                    <div id="group-role-proxy-checkbox" class="role-checkbox nf-checkbox checkbox-unchecked"></div><div class="role-name">Proxy</div>
-                </div>
-            </div>
-            <div class="clear"></div>
-        </div>
-    </div>
-</div>

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/users/user-delete-dialog.jsp
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/users/user-delete-dialog.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/users/user-delete-dialog.jsp
deleted file mode 100644
index d926691..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/users/user-delete-dialog.jsp
+++ /dev/null
@@ -1,23 +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.
---%>
-<%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %>
-<div id="user-delete-dialog">
-    <div class="dialog-content">
-        <input type="hidden" id="user-id-delete-dialog"/>
-        Are you sure you want to delete the user account for '<span id="user-name-delete-dialog"></span>'?
-    </div>
-</div>

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/users/user-details-dialog.jsp
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/users/user-details-dialog.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/users/user-details-dialog.jsp
deleted file mode 100644
index 8a81882..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/users/user-details-dialog.jsp
+++ /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.
---%>
-<%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %>
-<div id="user-details-dialog">
-    <div class="dialog-content">
-        <div class="setting">
-            <div class="setting-name">User</div>
-            <div class="setting-field">
-                <span id="user-name-details-dialog"></span>
-            </div>
-            <div class="clear"></div>
-        </div>
-        <div class="setting">
-            <div class="setting-name">Identity</div>
-            <div class="setting-field">
-                <span id="user-dn-details-dialog"></span>
-            </div>
-            <div class="clear"></div>
-        </div>
-        <div class="setting">
-            <div class="setting-name">Created</div>
-            <div class="setting-field">
-                <span id="user-created-details-dialog"></span>
-            </div>
-            <div class="clear"></div>
-        </div>
-        <div class="setting">
-            <div class="setting-name">Last Verified</div>
-            <div class="setting-field">
-                <span id="user-verified-details-dialog"></span>
-            </div>
-            <div class="clear"></div>
-        </div>
-        <div class="setting">
-            <div class="setting-name">Justification</div>
-            <div class="setting-field">
-                <div id="user-justification-details-dialog"></div>
-            </div>
-            <div class="clear"></div>
-        </div>
-    </div>
-</div>