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/11/12 21:38:45 UTC

nifi git commit: NIFI-655: - Persisting keys to sign user tokens. - Allowing the identity provider to specify the token expiration. - Code clean up.

Repository: nifi
Updated Branches:
  refs/heads/NIFI-655 b3ae3e314 -> 5ce44b1fe


NIFI-655:
- Persisting keys to sign user tokens.
- Allowing the identity provider to specify the token expiration.
- Code clean up.

Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/5ce44b1f
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/5ce44b1f
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/5ce44b1f

Branch: refs/heads/NIFI-655
Commit: 5ce44b1fe29fa9583c2b68b76bc1591af08104ee
Parents: b3ae3e3
Author: Matt Gilman <ma...@gmail.com>
Authored: Thu Nov 12 15:38:33 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Thu Nov 12 15:38:33 2015 -0500

----------------------------------------------------------------------
 .../authentication/AuthenticationResponse.java  |  46 ++++++
 .../authentication/LoginIdentityProvider.java   |  10 +-
 .../nifi/admin/KeyDataSourceFactoryBean.java    | 154 +++++++++++++++++++
 .../org/apache/nifi/admin/dao/DAOFactory.java   |   2 +
 .../java/org/apache/nifi/admin/dao/KeyDAO.java  |  39 +++++
 .../nifi/admin/dao/impl/DAOFactoryImpl.java     |   6 +
 .../nifi/admin/dao/impl/StandardKeyDAO.java     | 107 +++++++++++++
 .../apache/nifi/admin/service/KeyService.java   |  40 +++++
 .../nifi/admin/service/action/GetKeyAction.java |  41 +++++
 .../service/action/GetOrCreateKeyAction.java    |  47 ++++++
 .../admin/service/impl/StandardKeyService.java  | 125 +++++++++++++++
 .../resources/nifi-administration-context.xml   |  16 ++
 .../nifi/web/security/jwt/JwtService.java       |  48 ++++--
 .../login/LoginAuthenticationFilter.java        |  20 ++-
 .../LoginIdentityProviderFactoryBean.java       |  12 +-
 .../token/LoginAuthenticationToken.java         |  35 +++--
 .../resources/nifi-web-security-context.xml     |   2 +-
 .../nifi-ldap-iaa-providers/pom.xml             |   4 +
 .../apache/nifi/ldap/AbstractLdapProvider.java  |  27 +++-
 .../java/org/apache/nifi/ldap/LdapProvider.java |   2 +-
 20 files changed, 740 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/5ce44b1f/nifi-api/src/main/java/org/apache/nifi/authentication/AuthenticationResponse.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/authentication/AuthenticationResponse.java b/nifi-api/src/main/java/org/apache/nifi/authentication/AuthenticationResponse.java
new file mode 100644
index 0000000..8f57810
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/authentication/AuthenticationResponse.java
@@ -0,0 +1,46 @@
+/*
+ * 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.authentication;
+
+/**
+ * Authentication response for a user login attempt.
+ */
+public class AuthenticationResponse {
+
+    private final String identity;
+    private final String username;
+
+    /**
+     * Creates an authentication response. The username and how long the authentication is valid in milliseconds
+     *
+     * @param identity The user identity
+     * @param username The username
+     */
+    public AuthenticationResponse(final String identity, final String username) {
+        this.identity = identity;
+        this.username = username;
+    }
+
+    public String getIdentity() {
+        return identity;
+    }
+
+    public String getUsername() {
+        return username;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5ce44b1f/nifi-api/src/main/java/org/apache/nifi/authentication/LoginIdentityProvider.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/authentication/LoginIdentityProvider.java b/nifi-api/src/main/java/org/apache/nifi/authentication/LoginIdentityProvider.java
index 9039a4d..5f4db40 100644
--- a/nifi-api/src/main/java/org/apache/nifi/authentication/LoginIdentityProvider.java
+++ b/nifi-api/src/main/java/org/apache/nifi/authentication/LoginIdentityProvider.java
@@ -30,10 +30,18 @@ public interface LoginIdentityProvider {
      * Authenticates the specified login credentials.
      *
      * @param credentials the credentials
+     * @return The authentication response
      * @throws InvalidLoginCredentialsException The login credentials were invalid
      * @throws IdentityAccessException Unable to register the user due to an issue accessing the underlying storage
      */
-    void authenticate(LoginCredentials credentials) throws InvalidLoginCredentialsException, IdentityAccessException;
+    AuthenticationResponse authenticate(LoginCredentials credentials) throws InvalidLoginCredentialsException, IdentityAccessException;
+
+    /**
+     * Returns the expiration of a given authentication in milliseconds.
+     *
+     * @return The expiration in milliseconds
+     */
+    long getExpiration();
 
     /**
      * Called immediately after instance creation for implementers to perform additional setup

http://git-wip-us.apache.org/repos/asf/nifi/blob/5ce44b1f/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/KeyDataSourceFactoryBean.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/KeyDataSourceFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/KeyDataSourceFactoryBean.java
new file mode 100644
index 0000000..35c2a08
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/KeyDataSourceFactoryBean.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;
+
+import java.io.File;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import org.apache.commons.lang3.StringUtils;
+import org.h2.jdbcx.JdbcConnectionPool;
+import org.apache.nifi.util.NiFiProperties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.FactoryBean;
+
+/**
+ *
+ */
+public class KeyDataSourceFactoryBean implements FactoryBean {
+
+    private static final Logger logger = LoggerFactory.getLogger(KeyDataSourceFactoryBean.class);
+    private static final String NF_USERNAME_PASSWORD = "nf";
+    private static final int MAX_CONNECTIONS = 5;
+
+    // database file name
+    private static final String KEY_DATABASE_FILE_NAME = "nifi-key";
+
+    // ----------
+    // keys table
+    // ----------
+    private static final String CREATE_KEY_TABLE = "CREATE TABLE KEY ("
+            + "ID INT NOT NULL PRIMARY KEY AUTO_INCREMENT, "
+            + "IDENTITY VARCHAR2(4096) NOT NULL, "
+            + "KEY VARCHAR2(100) NOT NULL"
+            + ")";
+
+    private JdbcConnectionPool connectionPool;
+
+    private NiFiProperties properties;
+
+    @Override
+    public Object getObject() throws Exception {
+        if (connectionPool == null) {
+
+            // locate the repository directory
+            String repositoryDirectoryPath = properties.getProperty(NiFiProperties.REPOSITORY_DATABASE_DIRECTORY);
+
+            // ensure the repository directory is specified
+            if (repositoryDirectoryPath == null) {
+                throw new NullPointerException("Database directory must be specified.");
+            }
+
+            // create a handle to the repository directory
+            File repositoryDirectory = new File(repositoryDirectoryPath);
+
+            // get a handle to the database file
+            File databaseFile = new File(repositoryDirectory, KEY_DATABASE_FILE_NAME);
+
+            // format the database url
+            String databaseUrl = "jdbc:h2:" + databaseFile + ";AUTOCOMMIT=OFF;DB_CLOSE_ON_EXIT=FALSE;LOCK_MODE=3";
+            String databaseUrlAppend = properties.getProperty(NiFiProperties.H2_URL_APPEND);
+            if (StringUtils.isNotBlank(databaseUrlAppend)) {
+                databaseUrl += databaseUrlAppend;
+            }
+
+            // create the pool
+            connectionPool = JdbcConnectionPool.create(databaseUrl, NF_USERNAME_PASSWORD, NF_USERNAME_PASSWORD);
+            connectionPool.setMaxConnections(MAX_CONNECTIONS);
+
+            Connection connection = null;
+            ResultSet rs = null;
+            Statement statement = null;
+            try {
+                // get a connection
+                connection = connectionPool.getConnection();
+                connection.setAutoCommit(false);
+
+                // create a statement for initializing the database
+                statement = connection.createStatement();
+
+                // determine if the tables need to be created
+                rs = connection.getMetaData().getTables(null, null, "KEY", null);
+                if (!rs.next()) {
+                    logger.info("Database not built for repository: " + databaseUrl + ".  Building now...");
+                    RepositoryUtils.closeQuietly(rs);
+
+                    // action table
+                    statement.execute(CREATE_KEY_TABLE);
+                } else {
+                    logger.info("Existing database found and connected to at: " + databaseUrl);
+                }
+
+                // commit any changes
+                connection.commit();
+            } catch (SQLException sqle) {
+                RepositoryUtils.rollback(connection, logger);
+                throw sqle;
+            } finally {
+                RepositoryUtils.closeQuietly(rs);
+                RepositoryUtils.closeQuietly(statement);
+                RepositoryUtils.closeQuietly(connection);
+            }
+        }
+
+        return connectionPool;
+    }
+
+    @Override
+    public Class getObjectType() {
+        return JdbcConnectionPool.class;
+    }
+
+    @Override
+    public boolean isSingleton() {
+        return true;
+    }
+
+    public void setProperties(NiFiProperties properties) {
+        this.properties = properties;
+    }
+
+    /**
+     * Disposes resources.
+     */
+    public void shutdown() {
+
+        // shutdown the connection pool
+        if (connectionPool != null) {
+            try {
+                connectionPool.dispose();
+            } catch (Exception e) {
+                logger.warn("Unable to dispose of connection pool: " + e.getMessage());
+                if (logger.isDebugEnabled()) {
+                    logger.warn(StringUtils.EMPTY, e);
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5ce44b1f/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/DAOFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/DAOFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/DAOFactory.java
index dee4ef9..eb7e3ce 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/DAOFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/DAOFactory.java
@@ -26,4 +26,6 @@ public interface DAOFactory {
     ActionDAO getActionDAO();
 
     AuthorityDAO getAuthorityDAO();
+
+    KeyDAO getKeyDAO();
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5ce44b1f/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..0bc6e99
--- /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,39 @@
+/*
+ * 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;
+
+/**
+ * Key data access.
+ */
+public interface KeyDAO {
+
+    /**
+     * Gets the key for the specified user identity. Returns null if no key exists for the user identity.
+     *
+     * @param identity The user identity
+     * @return The key or null
+     */
+    String getKey(String identity);
+
+    /**
+     * Creates a key for the specified user identity.
+     *
+     * @param identity The user identity
+     * @return The key
+     */
+    String createKey(String identity);
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5ce44b1f/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/5ce44b1f/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..994e95b
--- /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,107 @@
+/*
+ * 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.util.UUID;
+import org.apache.nifi.admin.RepositoryUtils;
+import org.apache.nifi.admin.dao.DataAccessException;
+import org.apache.nifi.admin.dao.KeyDAO;
+
+/**
+ *
+ */
+public class StandardKeyDAO implements KeyDAO {
+
+    private static final String SELECT_KEY_FOR_USER = "SELECT 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 String getKey(String identity) {
+        if (identity == null) {
+            throw new IllegalArgumentException("Specified identity cannot be null.");
+        }
+
+        String key = null;
+
+        PreparedStatement statement = null;
+        ResultSet rs = null;
+        try {
+            // add each authority for the specified user
+            statement = connection.prepareStatement(SELECT_KEY_FOR_USER);
+            statement.setString(1, identity);
+
+            // execute the query
+            rs = statement.executeQuery();
+
+            // if the key was found, add it
+            if (rs.next()) {
+                key = rs.getString("KEY");
+            }
+        } catch (SQLException sqle) {
+            throw new DataAccessException(sqle);
+        } finally {
+            RepositoryUtils.closeQuietly(rs);
+            RepositoryUtils.closeQuietly(statement);
+        }
+
+        return key;
+    }
+
+    @Override
+    public String createKey(final String identity) {
+        PreparedStatement statement = null;
+        ResultSet rs = null;
+        try {
+            final String key = UUID.randomUUID().toString();
+
+            // add each authority for the specified user
+            statement = connection.prepareStatement(INSERT_KEY);
+            statement.setString(1, identity);
+            statement.setString(2, key);
+
+            // insert the key
+            int updateCount = statement.executeUpdate();
+            if (updateCount == 1) {
+                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/5ce44b1f/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..9346625
--- /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,40 @@
+/*
+ * 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;
+
+/**
+ * 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 identity The user identity
+     * @return The key or null
+     */
+    String getKey(String identity);
+
+    /**
+     * 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
+     */
+    String getOrCreateKey(String identity);
+}

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

http://git-wip-us.apache.org/repos/asf/nifi/blob/5ce44b1f/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..209cbcd
--- /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,47 @@
+/*
+ * 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;
+
+/**
+ * Gets a key for the specified user identity.
+ */
+public class GetOrCreateKeyAction implements AdministrationAction<String> {
+
+    private final String identity;
+
+    public GetOrCreateKeyAction(String identity) {
+        this.identity = identity;
+    }
+
+    @Override
+    public String execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) {
+        final KeyDAO keyDao = daoFactory.getKeyDAO();
+
+        String key = keyDao.getKey(identity);
+        if (key == null) {
+            key = keyDao.createKey(identity);
+        }
+
+        return key;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5ce44b1f/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..1b2f8c9
--- /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,125 @@
+/*
+ * 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.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.admin.service.KeyService;
+import org.apache.nifi.admin.service.action.GetKeyAction;
+import org.apache.nifi.admin.service.action.GetOrCreateKeyAction;
+
+/**
+ *
+ */
+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 String getKey(String identity) {
+        Transaction transaction = null;
+        String key = null;
+
+        readLock.lock();
+        try {
+            // start the transaction
+            transaction = transactionBuilder.start();
+
+            // seed the accounts
+            GetKeyAction addActions = new GetKeyAction(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);
+            readLock.unlock();
+        }
+
+        return key;
+    }
+
+    @Override
+    public String getOrCreateKey(String identity) {
+        Transaction transaction = null;
+        String 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/5ce44b1f/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/5ce44b1f/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/JwtService.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/jwt/JwtService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/JwtService.java
index 8afa15a..5dfbdfe 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/JwtService.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/JwtService.java
@@ -19,20 +19,23 @@ package org.apache.nifi.web.security.jwt;
 import io.jsonwebtoken.Claims;
 import io.jsonwebtoken.ExpiredJwtException;
 import io.jsonwebtoken.Jws;
+import io.jsonwebtoken.JwsHeader;
 import io.jsonwebtoken.Jwts;
 import io.jsonwebtoken.MalformedJwtException;
 import io.jsonwebtoken.SignatureAlgorithm;
 import io.jsonwebtoken.SignatureException;
+import io.jsonwebtoken.SigningKeyResolverAdapter;
 import io.jsonwebtoken.UnsupportedJwtException;
-import io.jsonwebtoken.impl.TextCodec;
 import java.io.IOException;
 import java.io.PrintWriter;
+import java.nio.charset.StandardCharsets;
 import java.util.Calendar;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.util.NiFiProperties;
-import org.springframework.security.core.Authentication;
+import org.apache.nifi.admin.service.AdministrationException;
+import org.apache.nifi.admin.service.KeyService;
+import org.apache.nifi.web.security.token.LoginAuthenticationToken;
 
 /**
  *
@@ -41,14 +44,10 @@ public class JwtService {
 
     private final static String AUTHORIZATION = "Authorization";
 
-    private final String key;
-    private final Integer expires;
+    private final KeyService keyService;
 
-    public JwtService(final NiFiProperties properties) {
-        // TODO - load key (and algo/provider?) and expiration from properties
-
-        key = TextCodec.BASE64.encode("nififtw!");
-        expires = 1;
+    public JwtService(final KeyService keyService) {
+        this.keyService = keyService;
     }
 
     /**
@@ -63,9 +62,22 @@ public class JwtService {
         final String token = StringUtils.substringAfterLast(authorization, " ");
 
         try {
-            final Jws<Claims> jwt = Jwts.parser().setSigningKey(key).parseClaimsJws(token);
+            final Jws<Claims> jwt = Jwts.parser().setSigningKeyResolver(new SigningKeyResolverAdapter() {
+                @Override
+                public byte[] resolveSigningKeyBytes(JwsHeader header, Claims claims) {
+                    final String identity = claims.getSubject();
+                    final String key = keyService.getKey(identity);
+
+                    // ensure we were able to find a key that was previously issued by this key service for this user
+                    if (key == null) {
+                        throw new UnsupportedJwtException("Unable to determine signing key for " + identity);
+                    }
+
+                    return key.getBytes(StandardCharsets.UTF_8);
+                }
+            }).parseClaimsJws(token);
             return jwt.getBody().getSubject();
-        } catch (final MalformedJwtException | UnsupportedJwtException | SignatureException | ExpiredJwtException | IllegalArgumentException e) {
+        } catch (final MalformedJwtException | UnsupportedJwtException | SignatureException | ExpiredJwtException | IllegalArgumentException | AdministrationException e) {
             return null;
         }
     }
@@ -77,15 +89,21 @@ public class JwtService {
      * @param authentication The authentication to generate a token for
      * @throws java.io.IOException if an io exception occurs
      */
-    public void addToken(final HttpServletResponse response, final Authentication authentication) throws IOException {
+    public void addToken(final HttpServletResponse response, final LoginAuthenticationToken authentication) throws IOException {
         // set expiration to one day from now
         final Calendar calendar = Calendar.getInstance();
-        calendar.add(Calendar.DATE, expires);
+        calendar.setTimeInMillis(calendar.getTimeInMillis() + authentication.getExpiration());
 
         // create a token the specified authentication
         final String identity = authentication.getPrincipal().toString();
         final String username = authentication.getName();
-        final String token = Jwts.builder().setSubject(identity).claim("preferred_username", username).setExpiration(calendar.getTime()).signWith(SignatureAlgorithm.HS512, key).compact();
+
+        // get/create the key for this user
+        final String key = keyService.getOrCreateKey(identity);
+        final byte[] keyBytes = key.getBytes(StandardCharsets.UTF_8);
+
+        // build the token
+        final String token = Jwts.builder().setSubject(identity).claim("preferred_username", username).setExpiration(calendar.getTime()).signWith(SignatureAlgorithm.HS512, keyBytes).compact();
 
         // add the token as a response header
         final PrintWriter out = response.getWriter();

http://git-wip-us.apache.org/repos/asf/nifi/blob/5ce44b1f/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/login/LoginAuthenticationFilter.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/login/LoginAuthenticationFilter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/login/LoginAuthenticationFilter.java
index 5c3b3e6..c90364a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/login/LoginAuthenticationFilter.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/login/LoginAuthenticationFilter.java
@@ -27,6 +27,8 @@ import javax.servlet.FilterChain;
 import javax.servlet.ServletException;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
+import org.apache.nifi.admin.service.AdministrationException;
+import org.apache.nifi.authentication.AuthenticationResponse;
 import org.apache.nifi.authentication.LoginCredentials;
 import org.apache.nifi.authentication.LoginIdentityProvider;
 import org.apache.nifi.authentication.exception.IdentityAccessException;
@@ -100,8 +102,7 @@ public class LoginAuthenticationFilter extends AbstractAuthenticationProcessingF
                     throw new AuthenticationCredentialsNotFoundException("Unable to issue token as issue token as no credentials were found in the request.");
                 }
 
-                final LoginCredentials tokenCredentials = new LoginCredentials(principal, null);
-                return new LoginAuthenticationToken(tokenCredentials);
+                return new LoginAuthenticationToken(principal, loginIdentityProvider.getExpiration());
             } else {
                 // extract the principal
                 final String principal = principalExtractor.extractPrincipal(certificate).toString();
@@ -133,8 +134,7 @@ public class LoginAuthenticationFilter extends AbstractAuthenticationProcessingF
                 // authorize the proxy if necessary
                 authorizeProxyIfNecessary(ProxiedEntitiesUtils.buildProxyChain(request, principal));
 
-                final LoginCredentials preAuthenticatedCredentials = new LoginCredentials(principal, null);
-                return new LoginAuthenticationToken(preAuthenticatedCredentials);
+                return new LoginAuthenticationToken(principal, loginIdentityProvider.getExpiration());
             }
         } else {
             // if not configuration for login, don't consider credentials
@@ -144,10 +144,10 @@ public class LoginAuthenticationFilter extends AbstractAuthenticationProcessingF
 
             try {
                 // attempt to authenticate
-                loginIdentityProvider.authenticate(credentials);
+                final AuthenticationResponse authenticationResponse = loginIdentityProvider.authenticate(credentials);
 
                 // create the authentication token
-                return new LoginAuthenticationToken(credentials);
+                return new LoginAuthenticationToken(authenticationResponse.getUsername(), loginIdentityProvider.getExpiration());
             } catch (final InvalidLoginCredentialsException ilce) {
                 throw new BadCredentialsException("The supplied username and password are not valid.", ilce);
             } catch (final IdentityAccessException iae) {
@@ -192,8 +192,12 @@ public class LoginAuthenticationFilter extends AbstractAuthenticationProcessingF
     protected void successfulAuthentication(final HttpServletRequest request, final HttpServletResponse response, final FilterChain chain, final Authentication authentication)
             throws IOException, ServletException {
 
-        // generate JWT for response
-        jwtService.addToken(response, authentication);
+        try {
+            // generate JWT for response
+            jwtService.addToken(response, (LoginAuthenticationToken) authentication);
+        } catch (final AdministrationException ae) {
+            unsuccessfulAuthentication(request, response, new AuthenticationServiceException(ae.getMessage(), ae));
+        }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/nifi/blob/5ce44b1f/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/spring/LoginIdentityProviderFactoryBean.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/spring/LoginIdentityProviderFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/spring/LoginIdentityProviderFactoryBean.java
index 0d35b29..e115b01 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/spring/LoginIdentityProviderFactoryBean.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/spring/LoginIdentityProviderFactoryBean.java
@@ -32,6 +32,7 @@ import javax.xml.transform.stream.StreamSource;
 import javax.xml.validation.Schema;
 import javax.xml.validation.SchemaFactory;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.authentication.AuthenticationResponse;
 import org.apache.nifi.authentication.LoginCredentials;
 import org.apache.nifi.authentication.LoginIdentityProvider;
 import org.apache.nifi.authentication.LoginIdentityProviderConfigurationContext;
@@ -259,9 +260,16 @@ public class LoginIdentityProviderFactoryBean implements FactoryBean, Disposable
         return new LoginIdentityProvider() {
 
             @Override
-            public void authenticate(LoginCredentials credentials) {
+            public AuthenticationResponse authenticate(LoginCredentials credentials) {
                 try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    baseProvider.authenticate(credentials);
+                    return baseProvider.authenticate(credentials);
+                }
+            }
+
+            @Override
+            public long getExpiration() {
+                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
+                    return baseProvider.getExpiration();
                 }
             }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/5ce44b1f/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/token/LoginAuthenticationToken.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/token/LoginAuthenticationToken.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/token/LoginAuthenticationToken.java
index f908d79..0bb0932 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/token/LoginAuthenticationToken.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/token/LoginAuthenticationToken.java
@@ -16,7 +16,6 @@
  */
 package org.apache.nifi.web.security.token;
 
-import org.apache.nifi.authentication.LoginCredentials;
 import org.apache.nifi.security.util.CertificateUtils;
 import org.springframework.security.authentication.AbstractAuthenticationToken;
 
@@ -25,32 +24,44 @@ import org.springframework.security.authentication.AbstractAuthenticationToken;
  */
 public class LoginAuthenticationToken extends AbstractAuthenticationToken {
 
-    final LoginCredentials credentials;
+    private final String identity;
+    private final String username;
+    private final long expiration;
 
-    public LoginAuthenticationToken(final LoginCredentials credentials) {
-        super(null);
-        setAuthenticated(true);
-        this.credentials = credentials;
+    public LoginAuthenticationToken(final String identity, final long expiration) {
+        this(identity, null, expiration);
     }
 
-    public LoginCredentials getLoginCredentials() {
-        return credentials;
+    public LoginAuthenticationToken(final String identity, final String username, final long expiration) {
+        super(null);
+        setAuthenticated(true);
+        this.identity = identity;
+        this.username = username;
+        this.expiration = expiration;
     }
 
     @Override
     public Object getCredentials() {
-        return credentials.getPassword();
+        return null;
     }
 
     @Override
     public Object getPrincipal() {
-        return credentials.getUsername();
+        return identity;
+    }
+
+    public long getExpiration() {
+        return expiration;
     }
 
     @Override
     public String getName() {
-        // if the username is a DN this will extract the username or CN... if not will return what was passed
-        return CertificateUtils.extractUsername(credentials.getUsername());
+        if (username == null) {
+            // if the username is a DN this will extract the username or CN... if not will return what was passed
+            return CertificateUtils.extractUsername(identity);
+        } else {
+            return username;
+        }
     }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5ce44b1f/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 fa0b5b8..0ffd46c 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
@@ -41,7 +41,7 @@
     
     <!-- jwt service -->
     <bean id="jwtService" class="org.apache.nifi.web.security.jwt.JwtService">
-        <constructor-arg ref="nifiProperties"/>
+        <constructor-arg ref="keyService"/>
     </bean>
     
     <!-- login identity provider -->

http://git-wip-us.apache.org/repos/asf/nifi/blob/5ce44b1f/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/pom.xml b/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/pom.xml
index 9f5b5e8..5cbd6f9 100644
--- a/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/pom.xml
+++ b/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/pom.xml
@@ -29,6 +29,10 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-security-utils</artifactId>
         </dependency>
         <dependency>

http://git-wip-us.apache.org/repos/asf/nifi/blob/5ce44b1f/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/AbstractLdapProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/AbstractLdapProvider.java b/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/AbstractLdapProvider.java
index c7f338b..3e8c52d 100644
--- a/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/AbstractLdapProvider.java
+++ b/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/AbstractLdapProvider.java
@@ -16,7 +16,9 @@
  */
 package org.apache.nifi.ldap;
 
+import java.util.concurrent.TimeUnit;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.authentication.AuthenticationResponse;
 import org.apache.nifi.authentication.LoginCredentials;
 import org.apache.nifi.authentication.LoginIdentityProvider;
 import org.apache.nifi.authentication.LoginIdentityProviderConfigurationContext;
@@ -25,11 +27,13 @@ import org.apache.nifi.authentication.exception.IdentityAccessException;
 import org.apache.nifi.authentication.exception.InvalidLoginCredentialsException;
 import org.apache.nifi.authorization.exception.ProviderCreationException;
 import org.apache.nifi.authorization.exception.ProviderDestructionException;
+import org.apache.nifi.util.FormatUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.security.authentication.AuthenticationServiceException;
 import org.springframework.security.authentication.BadCredentialsException;
 import org.springframework.security.authentication.UsernamePasswordAuthenticationToken;
+import org.springframework.security.core.Authentication;
 import org.springframework.security.ldap.authentication.AbstractLdapAuthenticationProvider;
 
 /**
@@ -40,6 +44,7 @@ public abstract class AbstractLdapProvider implements LoginIdentityProvider {
     private static final Logger logger = LoggerFactory.getLogger(AbstractLdapProvider.class);
 
     private AbstractLdapAuthenticationProvider provider;
+    private long expiration;
 
     @Override
     public final void initialize(final LoginIdentityProviderInitializationContext initializationContext) throws ProviderCreationException {
@@ -47,21 +52,32 @@ public abstract class AbstractLdapProvider implements LoginIdentityProvider {
 
     @Override
     public final void onConfigured(final LoginIdentityProviderConfigurationContext configurationContext) throws ProviderCreationException {
-        System.out.println(Thread.currentThread().getContextClassLoader());
+        final String rawExpiration = configurationContext.getProperty("Expiration Duration");
+        if (StringUtils.isBlank(rawExpiration)) {
+            throw new ProviderCreationException("The Expiration Duration must be specified.");
+        }
+
+        try {
+            expiration = FormatUtils.getTimeDuration(rawExpiration, TimeUnit.MILLISECONDS);
+        } catch (final NumberFormatException nfe) {
+            throw new ProviderCreationException(String.format("The Expiration Duration '%s' is not a valid time duration", rawExpiration));
+        }
+
         provider = getLdapAuthenticationProvider(configurationContext);
     }
 
     protected abstract AbstractLdapAuthenticationProvider getLdapAuthenticationProvider(LoginIdentityProviderConfigurationContext configurationContext) throws ProviderCreationException;
 
     @Override
-    public final void authenticate(final LoginCredentials credentials) throws InvalidLoginCredentialsException, IdentityAccessException {
+    public final AuthenticationResponse authenticate(final LoginCredentials credentials) throws InvalidLoginCredentialsException, IdentityAccessException {
         if (provider == null) {
             throw new IdentityAccessException("The LDAP authentication provider is not initialized.");
         }
 
         try {
             final UsernamePasswordAuthenticationToken token = new UsernamePasswordAuthenticationToken(credentials.getUsername(), credentials.getPassword());
-            provider.authenticate(token);
+            final Authentication authentication = provider.authenticate(token);
+            return new AuthenticationResponse(authentication.getPrincipal().toString(), credentials.getUsername());
         } catch (final AuthenticationServiceException ase) {
             logger.error(ase.getMessage());
             if (logger.isDebugEnabled()) {
@@ -74,6 +90,11 @@ public abstract class AbstractLdapProvider implements LoginIdentityProvider {
     }
 
     @Override
+    public long getExpiration() {
+        return expiration;
+    }
+
+    @Override
     public final void preDestruction() throws ProviderDestructionException {
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/5ce44b1f/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/LdapProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/LdapProvider.java b/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/LdapProvider.java
index cccff5d..7d2a7d9 100644
--- a/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/LdapProvider.java
+++ b/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/LdapProvider.java
@@ -45,7 +45,7 @@ import org.springframework.security.ldap.search.LdapUserSearch;
 public class LdapProvider extends AbstractLdapProvider {
 
     private static final String TLS = "TLS";
-    
+
     @Override
     protected AbstractLdapAuthenticationProvider getLdapAuthenticationProvider(LoginIdentityProviderConfigurationContext configurationContext) throws ProviderCreationException {
         final LdapContextSource context = new LdapContextSource();