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 2015/12/01 17:44:09 UTC

[31/51] [abbrv] nifi git commit: NIFI-655: - Refactoring web security to use Spring Security Java Configuration. - Introducing security in Web UI in order to get JWT.

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/KeyDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/KeyDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/KeyDAO.java
new file mode 100644
index 0000000..2a24e0b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/KeyDAO.java
@@ -0,0 +1,49 @@
+/*
+ * 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.dao;
+
+import org.apache.nifi.key.Key;
+
+/**
+ * Key data access.
+ */
+public interface KeyDAO {
+
+    /**
+     * Gets the key for the specified user identity. Returns null if no key exists for the key id.
+     *
+     * @param id The key id
+     * @return The key or null
+     */
+    Key findKeyById(int id);
+
+    /**
+     * Gets the latest key for the specified identity. Returns null if no key exists for the user identity.
+     *
+     * @param identity The identity
+     * @return The key or null
+     */
+    Key findLatestKeyByIdentity(String identity);
+
+    /**
+     * Creates a key for the specified user identity.
+     *
+     * @param identity The user identity
+     * @return The key
+     */
+    Key createKey(String identity);
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/DAOFactoryImpl.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/DAOFactoryImpl.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/DAOFactoryImpl.java
index 2f3de0e..940e364 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/DAOFactoryImpl.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/DAOFactoryImpl.java
@@ -20,6 +20,7 @@ import java.sql.Connection;
 import org.apache.nifi.admin.dao.ActionDAO;
 import org.apache.nifi.admin.dao.AuthorityDAO;
 import org.apache.nifi.admin.dao.DAOFactory;
+import org.apache.nifi.admin.dao.KeyDAO;
 import org.apache.nifi.admin.dao.UserDAO;
 
 /**
@@ -48,4 +49,9 @@ public class DAOFactoryImpl implements DAOFactory {
         return new StandardUserDAO(connection);
     }
 
+    @Override
+    public KeyDAO getKeyDAO() {
+        return new StandardKeyDAO(connection);
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java
index fe693e1..8fdfd34 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java
@@ -63,7 +63,7 @@ public class StandardActionDAO implements ActionDAO {
     // action table
     // ------------
     private static final String INSERT_ACTION = "INSERT INTO ACTION ("
-            + "USER_DN, USER_NAME, SOURCE_ID, SOURCE_NAME, SOURCE_TYPE, OPERATION, ACTION_TIMESTAMP"
+            + "IDENTITY, USER_NAME, SOURCE_ID, SOURCE_NAME, SOURCE_TYPE, OPERATION, ACTION_TIMESTAMP"
             + ") VALUES ("
             + "?, "
             + "?, "
@@ -216,8 +216,8 @@ public class StandardActionDAO implements ActionDAO {
         try {
             // obtain a statement to insert to the action table
             statement = connection.prepareStatement(INSERT_ACTION, Statement.RETURN_GENERATED_KEYS);
-            statement.setString(1, StringUtils.left(action.getUserIdentity(), 255));
-            statement.setString(2, StringUtils.left(action.getUserName(), 100));
+            statement.setString(1, StringUtils.left(action.getUserIdentity(), 4096));
+            statement.setString(2, StringUtils.left(action.getUserName(), 4096));
             statement.setString(3, action.getSourceId());
             statement.setString(4, StringUtils.left(action.getSourceName(), 1000));
             statement.setString(5, action.getSourceType().toString());
@@ -561,7 +561,7 @@ public class StandardActionDAO implements ActionDAO {
 
                 FlowChangeAction action = new FlowChangeAction();
                 action.setId(actionId);
-                action.setUserIdentity(rs.getString("USER_DN"));
+                action.setUserIdentity(rs.getString("IDENTITY"));
                 action.setUserName(rs.getString("USER_NAME"));
                 action.setOperation(Operation.valueOf(rs.getString("OPERATION")));
                 action.setTimestamp(new Date(rs.getTimestamp("ACTION_TIMESTAMP").getTime()));
@@ -635,7 +635,7 @@ public class StandardActionDAO implements ActionDAO {
                 // populate the action
                 action = new FlowChangeAction();
                 action.setId(rs.getInt("ID"));
-                action.setUserIdentity(rs.getString("USER_DN"));
+                action.setUserIdentity(rs.getString("IDENTITY"));
                 action.setUserName(rs.getString("USER_NAME"));
                 action.setOperation(operation);
                 action.setTimestamp(new Date(rs.getTimestamp("ACTION_TIMESTAMP").getTime()));

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardKeyDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardKeyDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardKeyDAO.java
new file mode 100644
index 0000000..f4bdc1d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardKeyDAO.java
@@ -0,0 +1,154 @@
+/*
+ * 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.dao.impl;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.UUID;
+import org.apache.nifi.admin.RepositoryUtils;
+import org.apache.nifi.admin.dao.DataAccessException;
+import org.apache.nifi.admin.dao.KeyDAO;
+import org.apache.nifi.key.Key;
+
+/**
+ *
+ */
+public class StandardKeyDAO implements KeyDAO {
+
+    private static final String SELECT_KEY_FOR_USER_BY_ID = "SELECT ID, IDENTITY, KEY "
+            + "FROM KEY "
+            + "WHERE ID = ?";
+
+    private static final String SELECT_KEY_FOR_USER_BY_IDENTITY = "SELECT ID, IDENTITY, KEY "
+            + "FROM KEY "
+            + "WHERE IDENTITY = ?";
+
+    private static final String INSERT_KEY = "INSERT INTO KEY ("
+            + "IDENTITY, KEY"
+            + ") VALUES ("
+            + "?, ?"
+            + ")";
+
+    private final Connection connection;
+
+    public StandardKeyDAO(Connection connection) {
+        this.connection = connection;
+    }
+
+    @Override
+    public Key findKeyById(int id) {
+        Key key = null;
+
+        PreparedStatement statement = null;
+        ResultSet rs = null;
+        try {
+            // add each authority for the specified user
+            statement = connection.prepareStatement(SELECT_KEY_FOR_USER_BY_ID);
+            statement.setInt(1, id);
+
+            // execute the query
+            rs = statement.executeQuery();
+
+            // if the key was found, add it
+            if (rs.next()) {
+                key = new Key();
+                key.setId(rs.getInt("ID"));
+                key.setIdentity(rs.getString("IDENTITY"));
+                key.setKey(rs.getString("KEY"));
+            }
+        } catch (SQLException sqle) {
+            throw new DataAccessException(sqle);
+        } finally {
+            RepositoryUtils.closeQuietly(rs);
+            RepositoryUtils.closeQuietly(statement);
+        }
+
+        return key;
+    }
+
+    @Override
+    public Key findLatestKeyByIdentity(String identity) {
+        if (identity == null) {
+            throw new IllegalArgumentException("Specified identity cannot be null.");
+        }
+
+        Key key = null;
+
+        PreparedStatement statement = null;
+        ResultSet rs = null;
+        try {
+            // add each authority for the specified user
+            statement = connection.prepareStatement(SELECT_KEY_FOR_USER_BY_IDENTITY);
+            statement.setString(1, identity);
+
+            // execute the query
+            rs = statement.executeQuery();
+
+            // if the key was found, add it
+            if (rs.next()) {
+                key = new Key();
+                key.setId(rs.getInt("ID"));
+                key.setIdentity(rs.getString("IDENTITY"));
+                key.setKey(rs.getString("KEY"));
+            }
+        } catch (SQLException sqle) {
+            throw new DataAccessException(sqle);
+        } finally {
+            RepositoryUtils.closeQuietly(rs);
+            RepositoryUtils.closeQuietly(statement);
+        }
+
+        return key;
+    }
+
+    @Override
+    public Key createKey(final String identity) {
+        PreparedStatement statement = null;
+        ResultSet rs = null;
+        try {
+            final String keyValue = UUID.randomUUID().toString();
+
+            // add each authority for the specified user
+            statement = connection.prepareStatement(INSERT_KEY, Statement.RETURN_GENERATED_KEYS);
+            statement.setString(1, identity);
+            statement.setString(2, keyValue);
+
+            // insert the key
+            int updateCount = statement.executeUpdate();
+            rs = statement.getGeneratedKeys();
+
+            // verify the results
+            if (updateCount == 1 && rs.next()) {
+                final Key key = new Key();
+                key.setId(rs.getInt(1));
+                key.setIdentity(identity);
+                key.setKey(keyValue);
+                return key;
+            } else {
+                throw new DataAccessException("Unable to add key for user.");
+            }
+        } catch (SQLException sqle) {
+            throw new DataAccessException(sqle);
+        } finally {
+            RepositoryUtils.closeQuietly(rs);
+            RepositoryUtils.closeQuietly(statement);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardUserDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardUserDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardUserDAO.java
index f6a62df..20356e3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardUserDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardUserDAO.java
@@ -45,9 +45,9 @@ public class StandardUserDAO implements UserDAO {
             + "FROM USER U "
             + "WHERE U.STATUS = 'PENDING'";
 
-    private static final String SELECT_USER_BY_DN = "SELECT "
+    private static final String SELECT_USER_BY_USER = "SELECT "
             + "U.ID, "
-            + "U.DN, "
+            + "U.IDENTITY, "
             + "U.USER_NAME, "
             + "U.USER_GROUP, "
             + "U.CREATION, "
@@ -59,11 +59,11 @@ public class StandardUserDAO implements UserDAO {
             + "FROM USER U "
             + "LEFT JOIN AUTHORITY A " // ensures that users without authorities are still matched
             + "ON U.ID = A.USER_ID "
-            + "WHERE U.DN = ?";
+            + "WHERE U.IDENTITY = ?";
 
     private static final String SELECT_USER_BY_ID = "SELECT "
             + "U.ID, "
-            + "U.DN, "
+            + "U.IDENTITY, "
             + "U.USER_NAME, "
             + "U.USER_GROUP, "
             + "U.CREATION, "
@@ -79,7 +79,7 @@ public class StandardUserDAO implements UserDAO {
 
     private static final String SELECT_USERS = "SELECT "
             + "U.ID, "
-            + "U.DN, "
+            + "U.IDENTITY, "
             + "U.USER_NAME, "
             + "U.USER_GROUP, "
             + "U.CREATION, "
@@ -91,7 +91,7 @@ public class StandardUserDAO implements UserDAO {
             + "FROM USER U "
             + "LEFT JOIN AUTHORITY A " // ensures that users without authorities are still matched
             + "ON U.ID = A.USER_ID "
-            + "WHERE U.DN <> ?";
+            + "WHERE U.IDENTITY <> ?";
 
     private static final String SELECT_USER_GROUPS = "SELECT DISTINCT "
             + "U.USER_GROUP "
@@ -99,7 +99,7 @@ public class StandardUserDAO implements UserDAO {
 
     private static final String SELECT_USER_GROUP = "SELECT "
             + "U.ID, "
-            + "U.DN, "
+            + "U.IDENTITY, "
             + "U.USER_NAME, "
             + "U.USER_GROUP, "
             + "U.CREATION, "
@@ -111,10 +111,10 @@ public class StandardUserDAO implements UserDAO {
             + "FROM USER U "
             + "LEFT JOIN AUTHORITY A " // ensures that users without authorities are still matched
             + "ON U.ID = A.USER_ID "
-            + "WHERE U.DN <> ? AND U.USER_GROUP = ?";
+            + "WHERE U.IDENTITY <> ? AND U.USER_GROUP = ?";
 
     private static final String INSERT_USER = "INSERT INTO USER ("
-            + "ID, DN, USER_NAME, USER_GROUP, CREATION, LAST_VERIFIED, JUSTIFICATION, STATUS"
+            + "ID, IDENTITY, USER_NAME, USER_GROUP, CREATION, LAST_VERIFIED, JUSTIFICATION, STATUS"
             + ") VALUES ("
             + "?, "
             + "?, "
@@ -127,7 +127,7 @@ public class StandardUserDAO implements UserDAO {
             + ")";
 
     private static final String UPDATE_USER = "UPDATE USER SET "
-            + "DN = ?, "
+            + "IDENTITY = ?, "
             + "USER_NAME = ?, "
             + "USER_GROUP = ?, "
             + "LAST_ACCESSED = ?, "
@@ -194,7 +194,7 @@ public class StandardUserDAO implements UserDAO {
         try {
             // create the connection and obtain a statement
             statement = connection.prepareStatement(SELECT_USERS);
-            statement.setString(1, NiFiUser.ANONYMOUS_USER_DN);
+            statement.setString(1, NiFiUser.ANONYMOUS_USER_IDENTITY);
 
             // execute the query
             rs = statement.executeQuery();
@@ -211,7 +211,7 @@ public class StandardUserDAO implements UserDAO {
                 if (user == null || !userId.equals(user.getId())) {
                     user = new NiFiUser();
                     user.setId(userId);
-                    user.setDn(rs.getString("DN"));
+                    user.setIdentity(rs.getString("IDENTITY"));
                     user.setUserName(rs.getString("USER_NAME"));
                     user.setUserGroup(rs.getString("USER_GROUP"));
                     user.setJustification(rs.getString("JUSTIFICATION"));
@@ -287,7 +287,7 @@ public class StandardUserDAO implements UserDAO {
         try {
             // create the connection and obtain a statement
             statement = connection.prepareStatement(SELECT_USER_GROUP);
-            statement.setString(1, NiFiUser.ANONYMOUS_USER_DN);
+            statement.setString(1, NiFiUser.ANONYMOUS_USER_IDENTITY);
             statement.setString(2, group);
 
             // execute the query
@@ -305,7 +305,7 @@ public class StandardUserDAO implements UserDAO {
                 if (user == null || !userId.equals(user.getId())) {
                     user = new NiFiUser();
                     user.setId(userId);
-                    user.setDn(rs.getString("DN"));
+                    user.setIdentity(rs.getString("IDENTITY"));
                     user.setUserName(rs.getString("USER_NAME"));
                     user.setUserGroup(rs.getString("USER_GROUP"));
                     user.setJustification(rs.getString("JUSTIFICATION"));
@@ -366,7 +366,7 @@ public class StandardUserDAO implements UserDAO {
                 if (user == null) {
                     user = new NiFiUser();
                     user.setId(rs.getString("ID"));
-                    user.setDn(rs.getString("DN"));
+                    user.setIdentity(rs.getString("IDENTITY"));
                     user.setUserName(rs.getString("USER_NAME"));
                     user.setUserGroup(rs.getString("USER_GROUP"));
                     user.setJustification(rs.getString("JUSTIFICATION"));
@@ -409,7 +409,7 @@ public class StandardUserDAO implements UserDAO {
         ResultSet rs = null;
         try {
             // create the connection and obtain a statement
-            statement = connection.prepareStatement(SELECT_USER_BY_DN);
+            statement = connection.prepareStatement(SELECT_USER_BY_USER);
             statement.setString(1, dn);
 
             // execute the query
@@ -424,7 +424,7 @@ public class StandardUserDAO implements UserDAO {
                 if (user == null) {
                     user = new NiFiUser();
                     user.setId(rs.getString("ID"));
-                    user.setDn(rs.getString("DN"));
+                    user.setIdentity(rs.getString("IDENTITY"));
                     user.setUserName(rs.getString("USER_NAME"));
                     user.setUserGroup(rs.getString("USER_GROUP"));
                     user.setJustification(rs.getString("JUSTIFICATION"));
@@ -463,20 +463,25 @@ public class StandardUserDAO implements UserDAO {
 
     @Override
     public NiFiUser createUser(NiFiUser user) throws DataAccessException {
-        if (user.getDn() == null) {
-            throw new IllegalArgumentException("User dn must be specified.");
+        if (user.getIdentity() == null) {
+            throw new IllegalArgumentException("User identity must be specified.");
+        }
+
+        // ensure the user identity is not too lengthy
+        if (user.getIdentity().length() > 4096) {
+            throw new IllegalArgumentException("User identity must be less than 4096 characters.");
         }
 
         PreparedStatement statement = null;
         ResultSet rs = null;
         try {
-            final String id = UUID.nameUUIDFromBytes(user.getDn().getBytes(StandardCharsets.UTF_8)).toString();
+            final String id = UUID.nameUUIDFromBytes(user.getIdentity().getBytes(StandardCharsets.UTF_8)).toString();
 
             // create a statement
             statement = connection.prepareStatement(INSERT_USER, Statement.RETURN_GENERATED_KEYS);
             statement.setString(1, id);
-            statement.setString(2, StringUtils.left(user.getDn(), 255));
-            statement.setString(3, StringUtils.left(user.getUserName(), 100));
+            statement.setString(2, StringUtils.left(user.getIdentity(), 4096));
+            statement.setString(3, StringUtils.left(user.getUserName(), 4096));
             statement.setString(4, StringUtils.left(user.getUserGroup(), 100));
             if (user.getLastVerified() != null) {
                 statement.setTimestamp(5, new java.sql.Timestamp(user.getLastVerified().getTime()));
@@ -531,8 +536,8 @@ public class StandardUserDAO implements UserDAO {
         try {
             // create a statement
             statement = connection.prepareStatement(UPDATE_USER);
-            statement.setString(1, StringUtils.left(user.getDn(), 255));
-            statement.setString(2, StringUtils.left(user.getUserName(), 100));
+            statement.setString(1, StringUtils.left(user.getIdentity(), 4096));
+            statement.setString(2, StringUtils.left(user.getUserName(), 4096));
             statement.setString(3, StringUtils.left(user.getUserGroup(), 100));
             statement.setString(6, StringUtils.left(user.getJustification(), 500));
             statement.setString(7, user.getStatus().toString());

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/KeyService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/KeyService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/KeyService.java
new file mode 100644
index 0000000..ae64c41
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/KeyService.java
@@ -0,0 +1,42 @@
+/*
+ * 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;
+
+import org.apache.nifi.key.Key;
+
+/**
+ * Supports retrieving and issues keys for signing user tokens.
+ */
+public interface KeyService {
+
+    /**
+     * Gets a key for the specified user identity. Returns null if the user has not had a key issued
+     *
+     * @param id The key id
+     * @return The key or null
+     */
+    Key getKey(int id);
+
+    /**
+     * Gets a key for the specified user identity. If a key does not exist, one will be created.
+     *
+     * @param identity The user identity
+     * @return The key
+     * @throws AdministrationException if it failed to get/create the key
+     */
+    Key getOrCreateKey(String identity);
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/AbstractUserAction.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/AbstractUserAction.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/AbstractUserAction.java
index b970dc1..69c6c1f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/AbstractUserAction.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/AbstractUserAction.java
@@ -76,14 +76,14 @@ public abstract class AbstractUserAction<T> implements AdministrationAction<T> {
      */
     protected void verifyAccount(AuthorityProvider authorityProvider, NiFiUser user) {
         // load the roles for the user
-        Set<Authority> authorities = authorityProvider.getAuthorities(user.getDn());
+        Set<Authority> authorities = authorityProvider.getAuthorities(user.getIdentity());
 
         // update the user's authorities
         user.getAuthorities().clear();
         user.getAuthorities().addAll(authorities);
 
         // get the user group
-        user.setUserGroup(authorityProvider.getGroupForUser(user.getDn()));
+        user.setUserGroup(authorityProvider.getGroupForUser(user.getIdentity()));
 
         // update the users status in case they were previously pending or disabled
         user.setStatus(AccountStatus.ACTIVE);

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/AuthorizeUserAction.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/AuthorizeUserAction.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/AuthorizeUserAction.java
index fe32772..ed4dfa1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/AuthorizeUserAction.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/AuthorizeUserAction.java
@@ -37,11 +37,11 @@ import org.apache.nifi.user.NiFiUser;
  */
 public class AuthorizeUserAction extends AbstractUserAction<NiFiUser> {
 
-    private final String dn;
+    private final String identity;
     private final int cacheDurationSeconds;
 
-    public AuthorizeUserAction(String dn, int cacheDurationSeconds) {
-        this.dn = dn;
+    public AuthorizeUserAction(String identity, int cacheDurationSeconds) {
+        this.identity = identity;
         this.cacheDurationSeconds = cacheDurationSeconds;
     }
 
@@ -50,14 +50,14 @@ public class AuthorizeUserAction extends AbstractUserAction<NiFiUser> {
         UserDAO userDao = daoFactory.getUserDAO();
 
         // get the user
-        NiFiUser user = userDao.findUserByDn(dn);
+        NiFiUser user = userDao.findUserByDn(identity);
 
         // verify the user was found
         if (user == null) {
             // determine whether this users exists
             boolean doesDnExist = false;
             try {
-                doesDnExist = authorityProvider.doesDnExist(dn);
+                doesDnExist = authorityProvider.doesDnExist(identity);
             } catch (AuthorityAccessException aae) {
                 throw new AdministrationException(String.format("Unable to access authority details: %s", aae.getMessage()), aae);
             }
@@ -66,8 +66,8 @@ public class AuthorizeUserAction extends AbstractUserAction<NiFiUser> {
             if (doesDnExist) {
                 // create the user
                 user = new NiFiUser();
-                user.setDn(dn);
-                user.setUserName(CertificateUtils.extractUsername(dn));
+                user.setIdentity(identity);
+                user.setUserName(CertificateUtils.extractUsername(identity));
                 user.setJustification("User details specified by authority provider.");
 
                 try {
@@ -86,12 +86,12 @@ public class AuthorizeUserAction extends AbstractUserAction<NiFiUser> {
                     createUser.execute(daoFactory, authorityProvider);
                 } catch (UnknownIdentityException uie) {
                     // strange since the provider just reported this dn existed but handleing anyways...
-                    throw new AccountNotFoundException(String.format("Unable to verify access for %s.", dn));
+                    throw new AccountNotFoundException(String.format("Unable to verify access for %s.", identity));
                 } catch (AuthorityAccessException aae) {
                     throw new AdministrationException(String.format("Unable to access authority details: %s", aae.getMessage()), aae);
                 }
             } else {
-                throw new AccountNotFoundException(String.format("Unable to verify access for %s.", dn));
+                throw new AccountNotFoundException(String.format("Unable to verify access for %s.", identity));
             }
         } else {
             Throwable providerError = null;
@@ -134,7 +134,7 @@ public class AuthorizeUserAction extends AbstractUserAction<NiFiUser> {
             updateUserAuthorities.execute(daoFactory, authorityProvider);
 
             if (providerError != null) {
-                throw new AccountDisabledException(String.format("User credentials for %s were not found. This account has been disabled.", user.getDn()), providerError);
+                throw new AccountDisabledException(String.format("User credentials for %s were not found. This account has been disabled.", user.getIdentity()), providerError);
             }
         }
 
@@ -165,9 +165,9 @@ public class AuthorizeUserAction extends AbstractUserAction<NiFiUser> {
      */
     private void checkAccountStatus(NiFiUser user) {
         if (AccountStatus.DISABLED.equals(user.getStatus())) {
-            throw new AccountDisabledException(String.format("Account for %s is disabled.", user.getDn()));
+            throw new AccountDisabledException(String.format("The account for %s has been disabled.", user.getIdentity()));
         } else if (AccountStatus.PENDING.equals(user.getStatus())) {
-            throw new AccountPendingException(String.format("Account for %s is pending.", user.getDn()));
+            throw new AccountPendingException(String.format("The account for %s is currently pending approval.", user.getIdentity()));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/DisableUserAction.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/DisableUserAction.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/DisableUserAction.java
index c31f107..9e9b798 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/DisableUserAction.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/DisableUserAction.java
@@ -63,12 +63,12 @@ public class DisableUserAction implements AdministrationAction<NiFiUser> {
 
         try {
             // revoke the user in the authority provider
-            authorityProvider.revokeUser(user.getDn());
+            authorityProvider.revokeUser(user.getIdentity());
         } catch (UnknownIdentityException uie) {
             // user identity is not known
-            logger.info(String.format("User %s has already been removed from the authority provider.", user.getDn()));
+            logger.info(String.format("User %s has already been removed from the authority provider.", user.getIdentity()));
         } catch (AuthorityAccessException aae) {
-            throw new AdministrationException(String.format("Unable to revoke user '%s': %s", user.getDn(), aae.getMessage()), aae);
+            throw new AdministrationException(String.format("Unable to revoke user '%s': %s", user.getIdentity(), aae.getMessage()), aae);
         }
 
         return user;

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetKeyByIdAction.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetKeyByIdAction.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetKeyByIdAction.java
new file mode 100644
index 0000000..8763b9d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetKeyByIdAction.java
@@ -0,0 +1,42 @@
+/*
+ * 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.authorization.AuthorityProvider;
+
+import org.apache.nifi.admin.dao.KeyDAO;
+import org.apache.nifi.key.Key;
+
+/**
+ * Gets a key for the specified key id.
+ */
+public class GetKeyByIdAction implements AdministrationAction<Key> {
+
+    private final int id;
+
+    public GetKeyByIdAction(int id) {
+        this.id = id;
+    }
+
+    @Override
+    public Key execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) {
+        final KeyDAO keyDao = daoFactory.getKeyDAO();
+        return keyDao.findKeyById(id);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetKeyByIdentityAction.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetKeyByIdentityAction.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetKeyByIdentityAction.java
new file mode 100644
index 0000000..9bcb0b3
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetKeyByIdentityAction.java
@@ -0,0 +1,42 @@
+/*
+ * 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.authorization.AuthorityProvider;
+
+import org.apache.nifi.admin.dao.KeyDAO;
+import org.apache.nifi.key.Key;
+
+/**
+ * Gets a key for the specified key id.
+ */
+public class GetKeyByIdentityAction implements AdministrationAction<Key> {
+
+    private final String identity;
+
+    public GetKeyByIdentityAction(String identity) {
+        this.identity = identity;
+    }
+
+    @Override
+    public Key execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) {
+        final KeyDAO keyDao = daoFactory.getKeyDAO();
+        return keyDao.findLatestKeyByIdentity(identity);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetOrCreateKeyAction.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetOrCreateKeyAction.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetOrCreateKeyAction.java
new file mode 100644
index 0000000..bb85b6f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetOrCreateKeyAction.java
@@ -0,0 +1,48 @@
+/*
+ * 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.authorization.AuthorityProvider;
+
+import org.apache.nifi.admin.dao.KeyDAO;
+import org.apache.nifi.key.Key;
+
+/**
+ * Gets a key for the specified user identity.
+ */
+public class GetOrCreateKeyAction implements AdministrationAction<Key> {
+
+    private final String identity;
+
+    public GetOrCreateKeyAction(String identity) {
+        this.identity = identity;
+    }
+
+    @Override
+    public Key execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) {
+        final KeyDAO keyDao = daoFactory.getKeyDAO();
+
+        Key key = keyDao.findLatestKeyByIdentity(identity);
+        if (key == null) {
+            key = keyDao.createKey(identity);
+        }
+
+        return key;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/RequestUserAccountAction.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/RequestUserAccountAction.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/RequestUserAccountAction.java
index 3dce6d9..198a32d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/RequestUserAccountAction.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/RequestUserAccountAction.java
@@ -30,11 +30,11 @@ import org.apache.nifi.user.NiFiUser;
  */
 public class RequestUserAccountAction implements AdministrationAction<NiFiUser> {
 
-    private final String dn;
+    private final String identity;
     private final String justification;
 
-    public RequestUserAccountAction(String dn, String justification) {
-        this.dn = dn;
+    public RequestUserAccountAction(String identity, String justification) {
+        this.identity = identity;
         this.justification = justification;
     }
 
@@ -43,15 +43,15 @@ public class RequestUserAccountAction implements AdministrationAction<NiFiUser>
         UserDAO userDao = daoFactory.getUserDAO();
 
         // determine if this user already exists
-        NiFiUser user = userDao.findUserByDn(dn);
+        NiFiUser user = userDao.findUserByDn(identity);
         if (user != null) {
-            throw new IllegalArgumentException(String.format("User account for %s already exists.", dn));
+            throw new IllegalArgumentException(String.format("User account for %s already exists.", identity));
         }
 
         // create the user
         user = new NiFiUser();
-        user.setDn(dn);
-        user.setUserName(CertificateUtils.extractUsername(dn));
+        user.setIdentity(identity);
+        user.setUserName(CertificateUtils.extractUsername(identity));
         user.setJustification(justification);
         user.setStatus(AccountStatus.PENDING);
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/SeedUserAccountsAction.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/SeedUserAccountsAction.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/SeedUserAccountsAction.java
index 6665ac0..c16cc71 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/SeedUserAccountsAction.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/SeedUserAccountsAction.java
@@ -46,7 +46,7 @@ public class SeedUserAccountsAction extends AbstractUserAction<Void> {
     @Override
     public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException {
         UserDAO userDao = daoFactory.getUserDAO();
-        Set<String> authorizedDns = new HashSet<>();
+        Set<String> authorizedIdentities = new HashSet<>();
 
         // get the current user cache
         final Set<NiFiUser> existingUsers;
@@ -62,7 +62,7 @@ public class SeedUserAccountsAction extends AbstractUserAction<Void> {
         try {
             // all users for all roles
             for (final Authority authority : Authority.values()) {
-                authorizedDns.addAll(authorityProvider.getUsers(authority));
+                authorizedIdentities.addAll(authorityProvider.getUsers(authority));
             }
         } catch (AuthorityAccessException aae) {
             // unable to access the authority provider... honor the cache
@@ -73,25 +73,25 @@ public class SeedUserAccountsAction extends AbstractUserAction<Void> {
         final Set<NiFiUser> accountsToRevoke = new HashSet<>(existingUsers);
 
         // persist the users
-        for (String dn : authorizedDns) {
+        for (String identity : authorizedIdentities) {
             NiFiUser user = null;
             try {
                 // locate the user for this dn
-                user = userDao.findUserByDn(dn);
+                user = userDao.findUserByDn(identity);
                 boolean newAccount = false;
 
                 // if the user does not exist, create a new account
                 if (user == null) {
-                    logger.info(String.format("Creating user account: %s", dn));
+                    logger.info(String.format("Creating user account: %s", identity));
                     newAccount = true;
 
                     // create the user
                     user = new NiFiUser();
-                    user.setDn(dn);
-                    user.setUserName(CertificateUtils.extractUsername(dn));
+                    user.setIdentity(identity);
+                    user.setUserName(CertificateUtils.extractUsername(identity));
                     user.setJustification("User details specified by authority provider.");
                 } else {
-                    logger.info(String.format("User account already created: %s. Updating authorities...", dn));
+                    logger.info(String.format("User account already created: %s. Updating authorities...", identity));
                 }
 
                 // verify the account
@@ -142,7 +142,7 @@ public class SeedUserAccountsAction extends AbstractUserAction<Void> {
             }
 
             try {
-                logger.info(String.format("User not authorized with configured provider: %s. Disabling account...", user.getDn()));
+                logger.info(String.format("User not authorized with configured provider: %s. Disabling account...", user.getIdentity()));
 
                 // disable the account and reset its last verified timestamp since it was not found
                 // in the current configured authority provider

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/UngroupUserAction.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/UngroupUserAction.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/UngroupUserAction.java
index 01eaf5f..2604a47 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/UngroupUserAction.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/UngroupUserAction.java
@@ -56,11 +56,11 @@ public class UngroupUserAction extends AbstractUserAction<Void> {
 
         try {
             // update the authority provider
-            authorityProvider.ungroupUser(user.getDn());
+            authorityProvider.ungroupUser(user.getIdentity());
         } catch (UnknownIdentityException uie) {
-            throw new AccountNotFoundException(String.format("Unable to ungroup user '%s': %s", user.getDn(), uie.getMessage()), uie);
+            throw new AccountNotFoundException(String.format("Unable to ungroup user '%s': %s", user.getIdentity(), uie.getMessage()), uie);
         } catch (AuthorityAccessException aae) {
-            throw new AdministrationException(String.format("Unable to ungroup user '%s': %s", user.getDn(), aae.getMessage()), aae);
+            throw new AdministrationException(String.format("Unable to ungroup user '%s': %s", user.getIdentity(), aae.getMessage()), aae);
         }
 
         return null;

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserAction.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserAction.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserAction.java
index cef21d7..ecb91e6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserAction.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserAction.java
@@ -61,41 +61,41 @@ public class UpdateUserAction extends AbstractUserAction<NiFiUser> {
         }
 
         // determine whether this users exists
-        boolean doesDnExist = false;
+        boolean doesIdentityExist = false;
         try {
-            doesDnExist = authorityProvider.doesDnExist(user.getDn());
+            doesIdentityExist = authorityProvider.doesDnExist(user.getIdentity());
         } catch (AuthorityAccessException aae) {
             throw new AdministrationException(String.format("Unable to access authority details: %s", aae.getMessage()), aae);
         }
 
         // if the user already doesn't exist, add them
-        if (!doesDnExist) {
+        if (!doesIdentityExist) {
             try {
                 // add the account account and group if necessary
-                authorityProvider.addUser(user.getDn(), user.getUserGroup());
+                authorityProvider.addUser(user.getIdentity(), user.getUserGroup());
             } catch (final IdentityAlreadyExistsException iaee) {
-                logger.warn(String.format("User '%s' already exists in the authority provider.  Continuing with user update.", user.getDn()));
+                logger.warn(String.format("User '%s' already exists in the authority provider.  Continuing with user update.", user.getIdentity()));
             } catch (AuthorityAccessException aae) {
-                throw new AdministrationException(String.format("Unable to access authorities for '%s': %s", user.getDn(), aae.getMessage()), aae);
+                throw new AdministrationException(String.format("Unable to access authorities for '%s': %s", user.getIdentity(), aae.getMessage()), aae);
             }
         }
 
         try {
             // update the authority provider as approprivate
-            authorityProvider.setAuthorities(user.getDn(), authorities);
+            authorityProvider.setAuthorities(user.getIdentity(), authorities);
         } catch (UnknownIdentityException uie) {
-            throw new AccountNotFoundException(String.format("Unable to modify authorities for '%s': %s.", user.getDn(), uie.getMessage()), uie);
+            throw new AccountNotFoundException(String.format("Unable to modify authorities for '%s': %s.", user.getIdentity(), uie.getMessage()), uie);
         } catch (AuthorityAccessException aae) {
-            throw new AdministrationException(String.format("Unable to access authorities for '%s': %s.", user.getDn(), aae.getMessage()), aae);
+            throw new AdministrationException(String.format("Unable to access authorities for '%s': %s.", user.getIdentity(), aae.getMessage()), aae);
         }
 
         try {
             // get the user group
-            user.setUserGroup(authorityProvider.getGroupForUser(user.getDn()));
+            user.setUserGroup(authorityProvider.getGroupForUser(user.getIdentity()));
         } catch (UnknownIdentityException uie) {
-            throw new AccountNotFoundException(String.format("Unable to determine the group for '%s': %s.", user.getDn(), uie.getMessage()), uie);
+            throw new AccountNotFoundException(String.format("Unable to determine the group for '%s': %s.", user.getIdentity(), uie.getMessage()), uie);
         } catch (AuthorityAccessException aae) {
-            throw new AdministrationException(String.format("Unable to access the group for '%s': %s.", user.getDn(), aae.getMessage()), aae);
+            throw new AdministrationException(String.format("Unable to access the group for '%s': %s.", user.getIdentity(), aae.getMessage()), aae);
         }
 
         // since all the authorities were updated accordingly, set the authorities

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserGroupAction.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserGroupAction.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserGroupAction.java
index 56b214c..1d7941f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserGroupAction.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserGroupAction.java
@@ -61,7 +61,7 @@ public class UpdateUserGroupAction extends AbstractUserAction<Void> {
 
         // record the new users being added to this group
         final Set<NiFiUser> newUsers = new HashSet<>();
-        final Set<String> newUserDns = new HashSet<>();
+        final Set<String> newUserIdentities = new HashSet<>();
 
         // if the user ids have been specified we need to create/update a group using the specified group name
         if (userIds != null) {
@@ -81,13 +81,13 @@ public class UpdateUserGroupAction extends AbstractUserAction<Void> {
 
                 try {
                     // if the user is unknown to the authority provider we cannot continue
-                    if (!authorityProvider.doesDnExist(user.getDn()) || AccountStatus.DISABLED.equals(user.getStatus())) {
-                        throw new IllegalStateException(String.format("Unable to group these users because access for '%s' is not %s.", user.getDn(), AccountStatus.ACTIVE.toString()));
+                    if (!authorityProvider.doesDnExist(user.getIdentity()) || AccountStatus.DISABLED.equals(user.getStatus())) {
+                        throw new IllegalStateException(String.format("Unable to group these users because access for '%s' is not %s.", user.getIdentity(), AccountStatus.ACTIVE.toString()));
                     }
 
                     // record the user being added to this group
                     newUsers.add(user);
-                    newUserDns.add(user.getDn());
+                    newUserIdentities.add(user.getIdentity());
                 } catch (final AuthorityAccessException aae) {
                     throw new AdministrationException(String.format("Unable to access authority details: %s", aae.getMessage()), aae);
                 }
@@ -95,11 +95,11 @@ public class UpdateUserGroupAction extends AbstractUserAction<Void> {
 
             try {
                 // update the authority provider
-                authorityProvider.setUsersGroup(newUserDns, group);
+                authorityProvider.setUsersGroup(newUserIdentities, group);
             } catch (UnknownIdentityException uie) {
-                throw new AccountNotFoundException(String.format("Unable to set user group '%s': %s", StringUtils.join(newUserDns, ", "), uie.getMessage()), uie);
+                throw new AccountNotFoundException(String.format("Unable to set user group '%s': %s", StringUtils.join(newUserIdentities, ", "), uie.getMessage()), uie);
             } catch (AuthorityAccessException aae) {
-                throw new AdministrationException(String.format("Unable to set user group '%s': %s", StringUtils.join(newUserDns, ", "), aae.getMessage()), aae);
+                throw new AdministrationException(String.format("Unable to set user group '%s': %s", StringUtils.join(newUserIdentities, ", "), aae.getMessage()), aae);
             }
         }
 
@@ -118,35 +118,35 @@ public class UpdateUserGroupAction extends AbstractUserAction<Void> {
             if (authorities != null) {
                 try {
                     // update the authority provider as approprivate
-                    authorityProvider.setAuthorities(user.getDn(), authorities);
+                    authorityProvider.setAuthorities(user.getIdentity(), authorities);
 
                     // since all the authorities were updated accordingly, set the authorities
                     user.getAuthorities().clear();
                     user.getAuthorities().addAll(authorities);
                 } catch (UnknownIdentityException uie) {
-                    throw new AccountNotFoundException(String.format("Unable to modify authorities for '%s': %s.", user.getDn(), uie.getMessage()), uie);
+                    throw new AccountNotFoundException(String.format("Unable to modify authorities for '%s': %s.", user.getIdentity(), uie.getMessage()), uie);
                 } catch (AuthorityAccessException aae) {
-                    throw new AdministrationException(String.format("Unable to access authorities for '%s': %s.", user.getDn(), aae.getMessage()), aae);
+                    throw new AdministrationException(String.format("Unable to access authorities for '%s': %s.", user.getIdentity(), aae.getMessage()), aae);
                 }
             } else {
                 try {
                     // refresh the authorities according to the provider
                     user.getAuthorities().clear();
-                    user.getAuthorities().addAll(authorityProvider.getAuthorities(user.getDn()));
+                    user.getAuthorities().addAll(authorityProvider.getAuthorities(user.getIdentity()));
                 } catch (UnknownIdentityException uie) {
-                    throw new AccountNotFoundException(String.format("Unable to determine the authorities for '%s': %s.", user.getDn(), uie.getMessage()), uie);
+                    throw new AccountNotFoundException(String.format("Unable to determine the authorities for '%s': %s.", user.getIdentity(), uie.getMessage()), uie);
                 } catch (AuthorityAccessException aae) {
-                    throw new AdministrationException(String.format("Unable to access authorities for '%s': %s.", user.getDn(), aae.getMessage()), aae);
+                    throw new AdministrationException(String.format("Unable to access authorities for '%s': %s.", user.getIdentity(), aae.getMessage()), aae);
                 }
             }
 
             try {
                 // get the user group
-                user.setUserGroup(authorityProvider.getGroupForUser(user.getDn()));
+                user.setUserGroup(authorityProvider.getGroupForUser(user.getIdentity()));
             } catch (UnknownIdentityException uie) {
-                throw new AccountNotFoundException(String.format("Unable to determine the group for '%s': %s.", user.getDn(), uie.getMessage()), uie);
+                throw new AccountNotFoundException(String.format("Unable to determine the group for '%s': %s.", user.getIdentity(), uie.getMessage()), uie);
             } catch (AuthorityAccessException aae) {
-                throw new AdministrationException(String.format("Unable to access the group for '%s': %s.", user.getDn(), aae.getMessage()), aae);
+                throw new AdministrationException(String.format("Unable to access the group for '%s': %s.", user.getIdentity(), aae.getMessage()), aae);
             }
 
             // update the users status in case they were previously pending or disabled

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/impl/StandardKeyService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/impl/StandardKeyService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/impl/StandardKeyService.java
new file mode 100644
index 0000000..ca0a124
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/impl/StandardKeyService.java
@@ -0,0 +1,126 @@
+/*
+ * 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.impl;
+
+import org.apache.nifi.admin.dao.DataAccessException;
+import org.apache.nifi.admin.service.AdministrationException;
+import org.apache.nifi.admin.service.KeyService;
+import org.apache.nifi.admin.service.action.GetKeyByIdAction;
+import org.apache.nifi.admin.service.action.GetOrCreateKeyAction;
+import org.apache.nifi.admin.service.transaction.Transaction;
+import org.apache.nifi.admin.service.transaction.TransactionBuilder;
+import org.apache.nifi.admin.service.transaction.TransactionException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.nifi.key.Key;
+
+/**
+ *
+ */
+public class StandardKeyService implements KeyService {
+
+    private static final Logger logger = LoggerFactory.getLogger(StandardKeyService.class);
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final ReentrantReadWriteLock.ReadLock readLock = lock.readLock();
+    private final ReentrantReadWriteLock.WriteLock writeLock = lock.writeLock();
+
+    private TransactionBuilder transactionBuilder;
+
+    @Override
+    public Key getKey(int id) {
+        Transaction transaction = null;
+        Key key = null;
+
+        readLock.lock();
+        try {
+            // start the transaction
+            transaction = transactionBuilder.start();
+
+            // get the key
+            GetKeyByIdAction addActions = new GetKeyByIdAction(id);
+            key = transaction.execute(addActions);
+
+            // commit the transaction
+            transaction.commit();
+        } catch (TransactionException | DataAccessException te) {
+            rollback(transaction);
+            throw new AdministrationException(te);
+        } catch (Throwable t) {
+            rollback(transaction);
+            throw t;
+        } finally {
+            closeQuietly(transaction);
+            readLock.unlock();
+        }
+
+        return key;
+    }
+
+    @Override
+    public Key getOrCreateKey(String identity) {
+        Transaction transaction = null;
+        Key key = null;
+
+        writeLock.lock();
+        try {
+            // start the transaction
+            transaction = transactionBuilder.start();
+
+            // seed the accounts
+            GetOrCreateKeyAction addActions = new GetOrCreateKeyAction(identity);
+            key = transaction.execute(addActions);
+
+            // commit the transaction
+            transaction.commit();
+        } catch (TransactionException | DataAccessException te) {
+            rollback(transaction);
+            throw new AdministrationException(te);
+        } catch (Throwable t) {
+            rollback(transaction);
+            throw t;
+        } finally {
+            closeQuietly(transaction);
+            writeLock.unlock();
+        }
+
+        return key;
+    }
+
+    private void rollback(Transaction transaction) {
+        if (transaction != null) {
+            transaction.rollback();
+        }
+    }
+
+    private void closeQuietly(final Transaction transaction) {
+        if (transaction != null) {
+            try {
+                transaction.close();
+            } catch (final IOException ioe) {
+            }
+        }
+    }
+
+    public void setTransactionBuilder(TransactionBuilder transactionBuilder) {
+        this.transactionBuilder = transactionBuilder;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/key/Key.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/key/Key.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/key/Key.java
new file mode 100644
index 0000000..9ce7a9a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/key/Key.java
@@ -0,0 +1,69 @@
+/*
+ * 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.key;
+
+import java.io.Serializable;
+
+/**
+ * An signing key for a NiFi user.
+ */
+public class Key implements Serializable {
+
+    private int id;
+    private String identity;
+    private String key;
+
+    /**
+     * The key id.
+     *
+     * @return the id
+     */
+    public int getId() {
+        return id;
+    }
+
+    public void setId(int id) {
+        this.id = id;
+    }
+
+    /**
+     * The identity of the user this key is associated with.
+     *
+     * @return the identity
+     */
+    public String getIdentity() {
+        return identity;
+    }
+
+    public void setIdentity(String identity) {
+        this.identity = identity;
+    }
+
+    /**
+     * The signing key.
+     *
+     * @return the signing key
+     */
+    public String getKey() {
+        return key;
+    }
+
+    public void setKey(String key) {
+        this.key = key;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/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 a47bde9..231b133 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
@@ -29,10 +29,10 @@ import org.apache.commons.lang3.StringUtils;
  */
 public class NiFiUser implements Serializable {
 
-    public static final String ANONYMOUS_USER_DN = "anonymous";
+    public static final String ANONYMOUS_USER_IDENTITY = "anonymous";
 
     private String id;
-    private String dn;
+    private String identity;
     private String userName;
     private String userGroup;
     private String justification;
@@ -55,12 +55,12 @@ public class NiFiUser implements Serializable {
         this.creation = creation;
     }
 
-    public String getDn() {
-        return dn;
+    public String getIdentity() {
+        return identity;
     }
 
-    public void setDn(String dn) {
-        this.dn = dn;
+    public void setIdentity(String identity) {
+        this.identity = identity;
     }
 
     public String getUserName() {
@@ -143,7 +143,7 @@ public class NiFiUser implements Serializable {
             return false;
         }
         final NiFiUser other = (NiFiUser) obj;
-        if (!Objects.equals(this.dn, other.dn)) {
+        if (!Objects.equals(this.identity, other.identity)) {
             return false;
         }
         return true;
@@ -152,13 +152,13 @@ public class NiFiUser implements Serializable {
     @Override
     public int hashCode() {
         int hash = 7;
-        hash = 53 * hash + Objects.hashCode(this.dn);
+        hash = 53 * hash + Objects.hashCode(this.identity);
         return hash;
     }
 
     @Override
     public String toString() {
-        return String.format("dn[%s], userName[%s], justification[%s], authorities[%s]", getDn(), getUserName(), getJustification(), StringUtils.join(getAuthorities(), ", "));
+        return String.format("identity[%s], userName[%s], justification[%s], authorities[%s]", getIdentity(), getUserName(), getJustification(), StringUtils.join(getAuthorities(), ", "));
     }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/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 8cb4b97..6d7b739 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
@@ -36,6 +36,11 @@
     <bean id="auditDataSource" class="org.apache.nifi.admin.AuditDataSourceFactoryBean" destroy-method="shutdown" depends-on="userDataSource">
         <property name="properties" ref="nifiProperties"/>
     </bean>
+    
+    <!-- initialize the data source -->
+    <bean id="keyDataSource" class="org.apache.nifi.admin.KeyDataSourceFactoryBean" 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">
@@ -48,6 +53,12 @@
         <property name="authorityProvider" ref="authorityProvider"/>
         <property name="dataSource" ref="auditDataSource"/>
     </bean>
+    
+    <!-- initialize the key transaction builder -->
+    <bean id="keyTransactionBuilder" class="org.apache.nifi.admin.service.transaction.impl.StandardTransactionBuilder">
+        <property name="authorityProvider" ref="authorityProvider"/>
+        <property name="dataSource" ref="keyDataSource"/>
+    </bean>
 
     <!-- administration service -->
     <bean id="userService" class="org.apache.nifi.admin.service.impl.StandardUserService" init-method="seedUserAccounts">
@@ -59,4 +70,9 @@
     <bean id="auditService" class="org.apache.nifi.admin.service.impl.StandardAuditService">
         <property name="transactionBuilder" ref="auditTransactionBuilder"/>
     </bean>
+
+    <!-- key service -->
+    <bean id="keyService" class="org.apache.nifi.admin.service.impl.StandardKeyService">
+        <property name="transactionBuilder" ref="keyTransactionBuilder"/>
+    </bean>
 </beans>

http://git-wip-us.apache.org/repos/asf/nifi/blob/aaf14c45/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
index 28ea4a9..8d3c15a 100644
--- 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
@@ -54,17 +54,17 @@ public class AuthorizeUserActionTest {
     private static final String USER_ID_10 = "10";
     private static final String USER_ID_11 = "11";
 
-    private static final String USER_DN_1 = "authority access exception while searching for user";
-    private static final String USER_DN_2 = "unknown user";
-    private static final String USER_DN_3 = "user removed after checking existence";
-    private static final String USER_DN_4 = "access exception getting authorities";
-    private static final String USER_DN_5 = "error creating user account";
-    private static final String USER_DN_6 = "create user general sequence";
-    private static final String USER_DN_7 = "existing user requires verification";
-    private static final String USER_DN_8 = "existing user does not require verification";
-    private static final String USER_DN_9 = "existing pending user";
-    private static final String USER_DN_10 = "existing disabled user";
-    private static final String USER_DN_11 = "existing user is now unknown in the authority provider";
+    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;
@@ -85,18 +85,18 @@ public class AuthorizeUserActionTest {
                 if (USER_ID_7.equals(id)) {
                     user = new NiFiUser();
                     user.setId(USER_ID_7);
-                    user.setDn(USER_DN_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.setDn(USER_DN_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.setDn(USER_DN_11);
+                    user.setIdentity(USER_IDENTITY_11);
                     user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR));
                     user.setStatus(AccountStatus.ACTIVE);
                 }
@@ -112,35 +112,35 @@ public class AuthorizeUserActionTest {
 
                 NiFiUser user = null;
                 switch (dn) {
-                    case USER_DN_7:
+                    case USER_IDENTITY_7:
                         user = new NiFiUser();
                         user.setId(USER_ID_7);
-                        user.setDn(USER_DN_7);
+                        user.setIdentity(USER_IDENTITY_7);
                         user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR));
                         break;
-                    case USER_DN_8:
+                    case USER_IDENTITY_8:
                         user = new NiFiUser();
                         user.setId(USER_ID_8);
-                        user.setDn(USER_DN_8);
+                        user.setIdentity(USER_IDENTITY_8);
                         user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR));
                         user.setLastVerified(new Date());
                         break;
-                    case USER_DN_9:
+                    case USER_IDENTITY_9:
                         user = new NiFiUser();
                         user.setId(USER_ID_9);
-                        user.setDn(USER_DN_9);
+                        user.setIdentity(USER_IDENTITY_9);
                         user.setStatus(AccountStatus.PENDING);
                         break;
-                    case USER_DN_10:
+                    case USER_IDENTITY_10:
                         user = new NiFiUser();
                         user.setId(USER_ID_10);
-                        user.setDn(USER_DN_10);
+                        user.setIdentity(USER_IDENTITY_10);
                         user.setStatus(AccountStatus.DISABLED);
                         break;
-                    case USER_DN_11:
+                    case USER_IDENTITY_11:
                         user = new NiFiUser();
                         user.setId(USER_ID_11);
-                        user.setDn(USER_DN_11);
+                        user.setIdentity(USER_IDENTITY_11);
                         user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR));
                         user.setStatus(AccountStatus.ACTIVE);
                         break;
@@ -154,10 +154,10 @@ public class AuthorizeUserActionTest {
             public Void answer(InvocationOnMock invocation) throws Throwable {
                 Object[] args = invocation.getArguments();
                 NiFiUser user = (NiFiUser) args[0];
-                switch (user.getDn()) {
-                    case USER_DN_5:
+                switch (user.getIdentity()) {
+                    case USER_IDENTITY_5:
                         throw new DataAccessException();
-                    case USER_DN_6:
+                    case USER_IDENTITY_6:
                         user.setId(USER_ID_6);
                         break;
                 }
@@ -215,9 +215,9 @@ public class AuthorizeUserActionTest {
                 Object[] args = invocation.getArguments();
                 String dn = (String) args[0];
                 switch (dn) {
-                    case USER_DN_1:
+                    case USER_IDENTITY_1:
                         throw new AuthorityAccessException(StringUtils.EMPTY);
-                    case USER_DN_2:
+                    case USER_IDENTITY_2:
                         return false;
                 }
 
@@ -231,21 +231,21 @@ public class AuthorizeUserActionTest {
                 String dn = (String) args[0];
                 Set<Authority> authorities = EnumSet.noneOf(Authority.class);
                 switch (dn) {
-                    case USER_DN_3:
+                    case USER_IDENTITY_3:
                         throw new UnknownIdentityException(StringUtils.EMPTY);
-                    case USER_DN_4:
+                    case USER_IDENTITY_4:
                         throw new AuthorityAccessException(StringUtils.EMPTY);
-                    case USER_DN_6:
+                    case USER_IDENTITY_6:
                         authorities.add(Authority.ROLE_MONITOR);
                         break;
-                    case USER_DN_7:
+                    case USER_IDENTITY_7:
                         authorities.add(Authority.ROLE_DFM);
                         break;
-                    case USER_DN_9:
+                    case USER_IDENTITY_9:
                         throw new UnknownIdentityException(StringUtils.EMPTY);
-                    case USER_DN_10:
+                    case USER_IDENTITY_10:
                         throw new UnknownIdentityException(StringUtils.EMPTY);
-                    case USER_DN_11:
+                    case USER_IDENTITY_11:
                         throw new UnknownIdentityException(StringUtils.EMPTY);
                 }
 
@@ -272,7 +272,7 @@ public class AuthorizeUserActionTest {
      */
     @Test(expected = AdministrationException.class)
     public void testAuthorityAccessExceptionInDoesDnExist() throws Exception {
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_1, 0);
+        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_IDENTITY_1, 0);
         authorizeUser.execute(daoFactory, authorityProvider);
     }
 
@@ -283,7 +283,7 @@ public class AuthorizeUserActionTest {
      */
     @Test(expected = AccountNotFoundException.class)
     public void testUnknownUser() throws Exception {
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_2, 0);
+        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_IDENTITY_2, 0);
         authorizeUser.execute(daoFactory, authorityProvider);
     }
 
@@ -294,7 +294,7 @@ public class AuthorizeUserActionTest {
      */
     @Test(expected = AccountNotFoundException.class)
     public void testUserRemovedAfterCheckingExistence() throws Exception {
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_3, 0);
+        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_IDENTITY_3, 0);
         authorizeUser.execute(daoFactory, authorityProvider);
     }
 
@@ -305,7 +305,7 @@ public class AuthorizeUserActionTest {
      */
     @Test(expected = AdministrationException.class)
     public void testAuthorityAccessException() throws Exception {
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_4, 0);
+        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_IDENTITY_4, 0);
         authorizeUser.execute(daoFactory, authorityProvider);
     }
 
@@ -316,7 +316,7 @@ public class AuthorizeUserActionTest {
      */
     @Test(expected = DataAccessException.class)
     public void testErrorCreatingUserAccount() throws Exception {
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_5, 0);
+        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_IDENTITY_5, 0);
         authorizeUser.execute(daoFactory, authorityProvider);
     }
 
@@ -327,11 +327,11 @@ public class AuthorizeUserActionTest {
      */
     @Test
     public void testAccountCreation() throws Exception {
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_6, 0);
+        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_IDENTITY_6, 0);
         NiFiUser user = authorizeUser.execute(daoFactory, authorityProvider);
 
         // verify the user
-        Assert.assertEquals(USER_DN_6, user.getDn());
+        Assert.assertEquals(USER_IDENTITY_6, user.getIdentity());
         Assert.assertEquals(1, user.getAuthorities().size());
         Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_MONITOR));
 
@@ -347,11 +347,11 @@ public class AuthorizeUserActionTest {
      */
     @Test
     public void testExistingUserRequiresVerification() throws Exception {
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_7, 0);
+        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_IDENTITY_7, 0);
         NiFiUser user = authorizeUser.execute(daoFactory, authorityProvider);
 
         // verify the user
-        Assert.assertEquals(USER_DN_7, user.getDn());
+        Assert.assertEquals(USER_IDENTITY_7, user.getIdentity());
         Assert.assertEquals(1, user.getAuthorities().size());
         Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_DFM));
 
@@ -369,11 +369,11 @@ public class AuthorizeUserActionTest {
     @Test
     public void testExistingUserNoVerification() throws Exception {
         // disabling verification by passing in a large cache duration
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_8, Integer.MAX_VALUE);
+        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_IDENTITY_8, Integer.MAX_VALUE);
         NiFiUser user = authorizeUser.execute(daoFactory, authorityProvider);
 
         // verify the user
-        Assert.assertEquals(USER_DN_8, user.getDn());
+        Assert.assertEquals(USER_IDENTITY_8, user.getIdentity());
         Assert.assertEquals(1, user.getAuthorities().size());
         Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_MONITOR));
 
@@ -391,7 +391,7 @@ public class AuthorizeUserActionTest {
     @Test(expected = AccountPendingException.class)
     public void testExistingPendingUser() throws Exception {
         // disabling verification by passing in a large cache duration
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_9, Integer.MAX_VALUE);
+        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_IDENTITY_9, Integer.MAX_VALUE);
         authorizeUser.execute(daoFactory, authorityProvider);
     }
 
@@ -403,7 +403,7 @@ public class AuthorizeUserActionTest {
     @Test(expected = AccountDisabledException.class)
     public void testExistingDisabledUser() throws Exception {
         // disabling verification by passing in a large cache duration
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_10, Integer.MAX_VALUE);
+        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_IDENTITY_10, Integer.MAX_VALUE);
         authorizeUser.execute(daoFactory, authorityProvider);
     }
 
@@ -416,7 +416,7 @@ public class AuthorizeUserActionTest {
     @Test
     public void testExistingActiveUserNotFoundInProvider() throws Exception {
         try {
-            AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_11, 0);
+            AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_IDENTITY_11, 0);
             authorizeUser.execute(daoFactory, authorityProvider);
 
             Assert.fail();