You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2021/07/31 11:46:37 UTC

[GitHub] [nifi] gresockj commented on a change in pull request #5262: NIFI-8766 Implemented RS512 Algorithm for JWT Signing

gresockj commented on a change in pull request #5262:
URL: https://github.com/apache/nifi/pull/5262#discussion_r680166368



##########
File path: nifi-docs/src/main/asciidoc/administration-guide.adoc
##########
@@ -489,6 +489,28 @@ To enable authentication via Apache Knox the following properties must be config
 this listing. The audience that is populated in the token can be configured in Knox.
 |==================================================================================================================================================
 
+[[json_web_token]]
+=== JSON Web Tokens
+
+NiFi uses JSON Web Tokens to provide authenticated access after the initial login process. Generated JSON Web Tokens include the authenticated user identity
+as well as the issuer and expiration from the configured Login Identity Provider.
+
+NiFi uses generated RSA Key Pairs with a key size of 4096 bits to support the `RS512` algorithm for JSON Web Signatures. The system stores RSA
+Public Keys using the configured local State Provider and retains the RSA Private Key in memory. This approach supports signature verification
+for the expiration configured in the Login Identity Provider without persisting the private key.
+
+JSON Web Token support includes revocation on logout using JSON Web Token Identifiers. The system denies access for expired tokens based on the
+Login Identity Provider configuration, but revocation invalidates the token prior to expiration. The system stores revoked identifiers using the

Review comment:
       I noticed that the SingleUserLoginIdentityProvider is hard-coded to 8 hours rather than being configured in login-identity-provider.xml as this implies.  I think that's appropriate for this provider, but you may want to clarify here.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/converter/StandardJwtAuthenticationConverter.java
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.security.jwt.converter;
+
+import org.apache.nifi.admin.service.IdpUserGroupService;
+import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.authorization.user.NiFiUserDetails;
+import org.apache.nifi.authorization.user.StandardNiFiUser;
+import org.apache.nifi.authorization.util.IdentityMapping;
+import org.apache.nifi.authorization.util.IdentityMappingUtil;
+import org.apache.nifi.authorization.util.UserGroupUtil;
+import org.apache.nifi.util.NiFiProperties;
+import org.apache.nifi.web.security.token.NiFiAuthenticationToken;
+import org.springframework.core.convert.converter.Converter;
+import org.springframework.security.oauth2.jwt.Jwt;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Standard Converter from JSON Web Token to NiFi Authentication Token
+ */
+public class StandardJwtAuthenticationConverter implements Converter<Jwt, NiFiAuthenticationToken> {
+    private final Authorizer authorizer;
+
+    private final IdpUserGroupService idpUserGroupService;
+
+    private final List<IdentityMapping> identityMappings;
+
+    public StandardJwtAuthenticationConverter(final Authorizer authorizer, final IdpUserGroupService idpUserGroupService, final NiFiProperties properties) {
+        this.authorizer = authorizer;
+        this.idpUserGroupService = idpUserGroupService;
+        this.identityMappings = IdentityMappingUtil.getIdentityMappings(properties);
+    }
+
+    /**
+     * Convert JSON Web Token to NiFi Authentication Token
+     *
+     * @param jwt JSON Web Token
+     * @return NiFi Authentication Token
+     */
+    @Override
+    public NiFiAuthenticationToken convert(final Jwt jwt) {
+        final NiFiUser user = getUser(jwt);
+        return new NiFiAuthenticationToken(new NiFiUserDetails(user));
+    }
+
+    private NiFiUser getUser(final Jwt jwt) {
+        final String identity = IdentityMappingUtil.mapIdentity(jwt.getSubject(), identityMappings);
+
+        return new StandardNiFiUser.Builder()
+                .identity(identity)
+                .groups(UserGroupUtil.getUserGroups(authorizer, identity))
+                .identityProviderGroups(getIdentityProviderGroups(identity))
+                .build();
+    }
+
+    private Set<String> getIdentityProviderGroups(final String identity) {
+        return idpUserGroupService.getUserGroups(identity).stream()
+                .map(userGroup -> userGroup.getGroupName())

Review comment:
       Minor suggestion:
   ```suggestion
                   .map(IdpUserGroup::getGroupName)
   ```

##########
File path: nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
##########
@@ -170,6 +171,7 @@
     public static final String SECURITY_GROUP_MAPPING_PATTERN_PREFIX = "nifi.security.group.mapping.pattern.";
     public static final String SECURITY_GROUP_MAPPING_VALUE_PREFIX = "nifi.security.group.mapping.value.";
     public static final String SECURITY_GROUP_MAPPING_TRANSFORM_PREFIX = "nifi.security.group.mapping.transform.";
+    public static final String SECURITY_USER_JWS_KEY_ROTATION_PERIOD = "nifi.security.user.jws.key.rotation.period";

Review comment:
       This should also be configured in `nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi.properties` and `nifi-assembly/pom.xml`

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/key/service/StandardVerificationKeyService.java
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.security.jwt.key.service;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.components.state.StateMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.security.Key;
+import java.security.KeyFactory;
+import java.security.NoSuchAlgorithmException;
+import java.security.spec.InvalidKeySpecException;
+import java.security.spec.KeySpec;
+import java.security.spec.X509EncodedKeySpec;
+import java.time.Instant;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/**
+ * Standard Verification Key Service implemented using State Manager
+ */
+public class StandardVerificationKeyService implements VerificationKeyService {
+    private static final Logger LOGGER = LoggerFactory.getLogger(StandardVerificationKeyService.class);
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper().registerModule(new JavaTimeModule());
+
+    private static final Scope SCOPE = Scope.LOCAL;
+
+    private final StateManager stateManager;
+
+    public StandardVerificationKeyService(final StateManager stateManager) {
+        this.stateManager = stateManager;
+    }
+
+    /**
+     * Find Key using specified Key Identifier
+     *
+     * @param id Key Identifier
+     * @return Optional Key
+     */
+    @Override
+    public Optional<Key> findById(final String id) {
+        final Optional<String> serializedKey = findSerializedKey(id);
+        return serializedKey.map(this::getVerificationKey).map(this::getKey);
+    }
+
+    /**
+     * Delete Expired Verification Keys is synchronized to avoid losing updates from other methods
+     */
+    @Override
+    public synchronized void deleteExpired() {
+        final Map<String, String> state = getStateMap().toMap();
+
+        final Instant now = Instant.now();
+        final Map<String, String> updatedState = state
+                .values()
+                .stream()
+                .map(this::getVerificationKey)
+                .filter(verificationKey -> verificationKey.getExpiration().isAfter(now))
+                .collect(Collectors.toMap(VerificationKey::getId, this::serializeVerificationKey));
+
+        if (updatedState.equals(state)) {
+            LOGGER.debug("Expired Verification Keys not found");
+        } else {
+            try {
+                stateManager.setState(updatedState, SCOPE);
+            } catch (final IOException e) {
+                throw new UncheckedIOException("Delete Expired Verification Keys Failed", e);
+            }
+            LOGGER.debug("Delete Expired Verification Keys: Before [{}] After [{}]", state.size(), updatedState.size());
+        }
+    }
+
+    /**
+     * Save Verification Key
+     *
+     * @param id Key Identifier
+     * @param key Key
+     * @param expiration Expiration
+     */
+    @Override
+    public void save(final String id, final Key key, final Instant expiration) {
+        final VerificationKey verificationKey = new VerificationKey();
+        verificationKey.setId(id);
+        verificationKey.setEncoded(key.getEncoded());
+        verificationKey.setAlgorithm(key.getAlgorithm());
+        verificationKey.setExpiration(expiration);
+        setVerificationKey(verificationKey);
+    }
+
+    /**
+     * Set Expiration of Verification Key when found
+     *
+     * @param id Key Identifier
+     * @param expiration Expiration
+     */
+    @Override
+    public void setExpiration(final String id, final Instant expiration) {
+        final Optional<String> serializedKey = findSerializedKey(id);
+        if (serializedKey.isPresent()) {
+            final VerificationKey verificationKey = getVerificationKey(serializedKey.get());
+            verificationKey.setExpiration(expiration);
+            setVerificationKey(verificationKey);
+        }
+    }
+
+    /**
+     * Set Verification Key is synchronized to avoid competing updates to the State Map
+     *
+     * @param verificationKey Verification Key to be stored
+     */
+    private synchronized void setVerificationKey(final VerificationKey verificationKey) {
+        try {
+            final String serialized = serializeVerificationKey(verificationKey);
+            final Map<String, String> state = new HashMap<>(getStateMap().toMap());
+            state.put(verificationKey.getId(), serialized);
+            stateManager.setState(state, SCOPE);
+        } catch (final IOException e) {
+            throw new UncheckedIOException("Set Verification Key State Failed", e);
+        }
+        LOGGER.debug("Stored Verification Key [{}] Expiration [{}]", verificationKey.getId(), verificationKey.getExpiration());
+    }
+
+    private Optional<String> findSerializedKey(final String id) {
+        final StateMap stateMap = getStateMap();
+        final String serializedKey = stateMap.get(id);
+        return serializedKey == null ? Optional.empty() : Optional.of(serializedKey);

Review comment:
       Could this be:
   ```suggestion
           return Optional.ofNullable(stateMap.get(id));
   ```

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/jws/JwsSignerProvider.java
##########
@@ -14,31 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.admin.service.action;
+package org.apache.nifi.web.security.jwt.jws;
 
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.KeyDAO;
+import java.time.Instant;
 
 /**
- *
+ * JSON Web Signature Signer Provider supports accessing signer and identifier properties
  */
-public class DeleteKeyAction implements AdministrationAction<Integer> {

Review comment:
       A bit perplexing which classes GitHub sometimes thinks are the same, no? 🙂 

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/key/VerificationKeySelector.java
##########
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.security.jwt.key;
+
+import java.security.Key;
+import java.util.List;
+
+/**
+ * Verification Key Selector returns a List of java.security.Key objects for signature verification
+ */
+public interface VerificationKeySelector {
+    /**
+     * Get Verification Keys for Key Identifier
+     *
+     * @param keyIdentifier Key Identifier
+     * @return List of Keys

Review comment:
       I liked the description from from `StandardJWSKeySelector`:
   ```suggestion
        * @return List of found java.security.Key objects
   ```

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/configuration/JwtAuthenticationSecurityConfiguration.java
##########
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.security.configuration;
+
+import com.nimbusds.jose.proc.JWSKeySelector;
+import com.nimbusds.jose.proc.SecurityContext;
+import com.nimbusds.jwt.proc.DefaultJWTClaimsVerifier;
+import com.nimbusds.jwt.proc.DefaultJWTProcessor;
+import com.nimbusds.jwt.proc.JWTClaimsSetVerifier;
+import com.nimbusds.jwt.proc.JWTProcessor;
+import org.apache.nifi.admin.service.IdpUserGroupService;
+import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.components.state.StateManagerProvider;
+import org.apache.nifi.util.NiFiProperties;
+import org.apache.nifi.web.security.jwt.converter.StandardJwtAuthenticationConverter;
+import org.apache.nifi.web.security.jwt.jws.StandardJWSKeySelector;
+import org.apache.nifi.web.security.jwt.jws.StandardJwsSignerProvider;
+import org.apache.nifi.web.security.jwt.key.command.KeyExpirationCommand;
+import org.apache.nifi.web.security.jwt.key.command.KeyGenerationCommand;
+import org.apache.nifi.web.security.jwt.key.StandardVerificationKeySelector;
+import org.apache.nifi.web.security.jwt.key.service.StandardVerificationKeyService;
+import org.apache.nifi.web.security.jwt.key.service.VerificationKeyService;
+import org.apache.nifi.web.security.jwt.provider.BearerTokenProvider;
+import org.apache.nifi.web.security.jwt.provider.StandardBearerTokenProvider;
+import org.apache.nifi.web.security.jwt.revocation.JwtLogoutListener;
+import org.apache.nifi.web.security.jwt.revocation.JwtRevocationService;
+import org.apache.nifi.web.security.jwt.revocation.JwtRevocationValidator;
+import org.apache.nifi.web.security.jwt.revocation.StandardJwtLogoutListener;
+import org.apache.nifi.web.security.jwt.revocation.StandardJwtRevocationService;
+import org.apache.nifi.web.security.jwt.revocation.command.RevocationExpirationCommand;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.scheduling.concurrent.ThreadPoolTaskScheduler;
+import org.springframework.security.oauth2.core.DelegatingOAuth2TokenValidator;
+import org.springframework.security.oauth2.core.OAuth2TokenValidator;
+import org.springframework.security.oauth2.jwt.Jwt;
+import org.springframework.security.oauth2.jwt.JwtDecoder;
+import org.springframework.security.oauth2.jwt.JwtValidators;
+import org.springframework.security.oauth2.jwt.NimbusJwtDecoder;
+import org.springframework.security.oauth2.server.resource.authentication.JwtAuthenticationProvider;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * JSON Web Token Configuration for Authentication Security
+ */
+@Configuration
+public class JwtAuthenticationSecurityConfiguration {
+    private static final Set<String> REQUIRED_CLAIMS = new HashSet<>(Arrays.asList("sub", "iss", "aud", "nbf", "iat", "exp", "jti"));
+
+    private final NiFiProperties niFiProperties;
+
+    private final Authorizer authorizer;
+
+    private final IdpUserGroupService idpUserGroupService;
+
+    private final StateManagerProvider stateManagerProvider;
+
+    @Autowired
+    public JwtAuthenticationSecurityConfiguration(
+            final NiFiProperties niFiProperties,
+            final Authorizer authorizer,
+            final IdpUserGroupService idpUserGroupService,
+            final StateManagerProvider stateManagerProvider
+    ) {
+        this.niFiProperties = niFiProperties;
+        this.authorizer = authorizer;
+        this.idpUserGroupService = idpUserGroupService;
+        this.stateManagerProvider = stateManagerProvider;
+    }
+
+    @Bean
+    public JwtAuthenticationProvider jwtAuthenticationProvider() {
+        final JwtAuthenticationProvider jwtAuthenticationProvider = new JwtAuthenticationProvider(jwtDecoder());
+        jwtAuthenticationProvider.setJwtAuthenticationConverter(jwtAuthenticationConverter());
+        return jwtAuthenticationProvider;
+    }
+
+    @Bean
+    public JwtDecoder jwtDecoder() {
+        final NimbusJwtDecoder jwtDecoder = new NimbusJwtDecoder(jwtProcessor());
+        final OAuth2TokenValidator<Jwt> jwtValidator = new DelegatingOAuth2TokenValidator<>(
+                JwtValidators.createDefault(),
+                jwtRevocationValidator()
+        );
+        jwtDecoder.setJwtValidator(jwtValidator);
+        return jwtDecoder;
+    }
+
+    @Bean
+    public OAuth2TokenValidator<Jwt> jwtRevocationValidator() {
+        return new JwtRevocationValidator(jwtRevocationService());
+    }
+
+    @Bean
+    public JwtRevocationService jwtRevocationService() {
+        final StateManager stateManager = stateManagerProvider.getStateManager(StandardJwtRevocationService.class.getName());
+        return new StandardJwtRevocationService(stateManager);
+    }
+
+    @Bean
+    public JwtLogoutListener jwtLogoutListener() {
+        return new StandardJwtLogoutListener(jwtDecoder(), jwtRevocationService());
+    }
+
+    @Bean
+    public JWTProcessor<SecurityContext> jwtProcessor() {
+        final DefaultJWTProcessor<SecurityContext> jwtProcessor = new DefaultJWTProcessor<>();
+        jwtProcessor.setJWSKeySelector(jwsKeySelector());
+        jwtProcessor.setJWTClaimsSetVerifier(claimsSetVerifier());
+        return jwtProcessor;
+    }
+
+    @Bean
+    public JWSKeySelector<SecurityContext> jwsKeySelector() {
+        return new StandardJWSKeySelector<>(verificationKeySelector());
+    }
+
+    @Bean
+    public JWTClaimsSetVerifier<SecurityContext> claimsSetVerifier() {
+        return new DefaultJWTClaimsVerifier<>(null, REQUIRED_CLAIMS);
+    }
+
+    @Bean
+    public StandardJwtAuthenticationConverter jwtAuthenticationConverter() {
+        return new StandardJwtAuthenticationConverter(authorizer, idpUserGroupService, niFiProperties);
+    }
+
+    @Bean
+    public BearerTokenProvider bearerTokenProvider() {
+        return new StandardBearerTokenProvider(jwsSignerProvider());
+    }
+
+    @Bean
+    public StandardJwsSignerProvider jwsSignerProvider() {
+        return new StandardJwsSignerProvider(verificationKeySelector());
+    }
+
+    @Bean
+    public StandardVerificationKeySelector verificationKeySelector() {
+        return new StandardVerificationKeySelector(verificationKeyService(), niFiProperties.getSecurityUserJwsKeyRotationPeriod());

Review comment:
       Do you think it would be appropriate to store the value of `niFiProperties.getSecurityUserJwsKeyRotationPeriod()` in an instance variable so it can be reused among these methods?

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/configuration/JwtAuthenticationSecurityConfiguration.java
##########
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.security.configuration;
+
+import com.nimbusds.jose.proc.JWSKeySelector;
+import com.nimbusds.jose.proc.SecurityContext;
+import com.nimbusds.jwt.proc.DefaultJWTClaimsVerifier;
+import com.nimbusds.jwt.proc.DefaultJWTProcessor;
+import com.nimbusds.jwt.proc.JWTClaimsSetVerifier;
+import com.nimbusds.jwt.proc.JWTProcessor;
+import org.apache.nifi.admin.service.IdpUserGroupService;
+import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.components.state.StateManagerProvider;
+import org.apache.nifi.util.NiFiProperties;
+import org.apache.nifi.web.security.jwt.converter.StandardJwtAuthenticationConverter;
+import org.apache.nifi.web.security.jwt.jws.StandardJWSKeySelector;
+import org.apache.nifi.web.security.jwt.jws.StandardJwsSignerProvider;
+import org.apache.nifi.web.security.jwt.key.command.KeyExpirationCommand;
+import org.apache.nifi.web.security.jwt.key.command.KeyGenerationCommand;
+import org.apache.nifi.web.security.jwt.key.StandardVerificationKeySelector;
+import org.apache.nifi.web.security.jwt.key.service.StandardVerificationKeyService;
+import org.apache.nifi.web.security.jwt.key.service.VerificationKeyService;
+import org.apache.nifi.web.security.jwt.provider.BearerTokenProvider;
+import org.apache.nifi.web.security.jwt.provider.StandardBearerTokenProvider;
+import org.apache.nifi.web.security.jwt.revocation.JwtLogoutListener;
+import org.apache.nifi.web.security.jwt.revocation.JwtRevocationService;
+import org.apache.nifi.web.security.jwt.revocation.JwtRevocationValidator;
+import org.apache.nifi.web.security.jwt.revocation.StandardJwtLogoutListener;
+import org.apache.nifi.web.security.jwt.revocation.StandardJwtRevocationService;
+import org.apache.nifi.web.security.jwt.revocation.command.RevocationExpirationCommand;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.scheduling.concurrent.ThreadPoolTaskScheduler;
+import org.springframework.security.oauth2.core.DelegatingOAuth2TokenValidator;
+import org.springframework.security.oauth2.core.OAuth2TokenValidator;
+import org.springframework.security.oauth2.jwt.Jwt;
+import org.springframework.security.oauth2.jwt.JwtDecoder;
+import org.springframework.security.oauth2.jwt.JwtValidators;
+import org.springframework.security.oauth2.jwt.NimbusJwtDecoder;
+import org.springframework.security.oauth2.server.resource.authentication.JwtAuthenticationProvider;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * JSON Web Token Configuration for Authentication Security
+ */
+@Configuration
+public class JwtAuthenticationSecurityConfiguration {
+    private static final Set<String> REQUIRED_CLAIMS = new HashSet<>(Arrays.asList("sub", "iss", "aud", "nbf", "iat", "exp", "jti"));
+
+    private final NiFiProperties niFiProperties;
+
+    private final Authorizer authorizer;
+
+    private final IdpUserGroupService idpUserGroupService;
+
+    private final StateManagerProvider stateManagerProvider;
+
+    @Autowired
+    public JwtAuthenticationSecurityConfiguration(
+            final NiFiProperties niFiProperties,
+            final Authorizer authorizer,
+            final IdpUserGroupService idpUserGroupService,
+            final StateManagerProvider stateManagerProvider
+    ) {
+        this.niFiProperties = niFiProperties;
+        this.authorizer = authorizer;
+        this.idpUserGroupService = idpUserGroupService;
+        this.stateManagerProvider = stateManagerProvider;
+    }
+
+    @Bean
+    public JwtAuthenticationProvider jwtAuthenticationProvider() {
+        final JwtAuthenticationProvider jwtAuthenticationProvider = new JwtAuthenticationProvider(jwtDecoder());
+        jwtAuthenticationProvider.setJwtAuthenticationConverter(jwtAuthenticationConverter());
+        return jwtAuthenticationProvider;
+    }
+
+    @Bean
+    public JwtDecoder jwtDecoder() {
+        final NimbusJwtDecoder jwtDecoder = new NimbusJwtDecoder(jwtProcessor());
+        final OAuth2TokenValidator<Jwt> jwtValidator = new DelegatingOAuth2TokenValidator<>(
+                JwtValidators.createDefault(),
+                jwtRevocationValidator()
+        );
+        jwtDecoder.setJwtValidator(jwtValidator);
+        return jwtDecoder;
+    }
+
+    @Bean
+    public OAuth2TokenValidator<Jwt> jwtRevocationValidator() {
+        return new JwtRevocationValidator(jwtRevocationService());
+    }
+
+    @Bean
+    public JwtRevocationService jwtRevocationService() {
+        final StateManager stateManager = stateManagerProvider.getStateManager(StandardJwtRevocationService.class.getName());
+        return new StandardJwtRevocationService(stateManager);
+    }
+
+    @Bean
+    public JwtLogoutListener jwtLogoutListener() {
+        return new StandardJwtLogoutListener(jwtDecoder(), jwtRevocationService());
+    }
+
+    @Bean
+    public JWTProcessor<SecurityContext> jwtProcessor() {
+        final DefaultJWTProcessor<SecurityContext> jwtProcessor = new DefaultJWTProcessor<>();
+        jwtProcessor.setJWSKeySelector(jwsKeySelector());
+        jwtProcessor.setJWTClaimsSetVerifier(claimsSetVerifier());
+        return jwtProcessor;
+    }
+
+    @Bean
+    public JWSKeySelector<SecurityContext> jwsKeySelector() {
+        return new StandardJWSKeySelector<>(verificationKeySelector());
+    }
+
+    @Bean
+    public JWTClaimsSetVerifier<SecurityContext> claimsSetVerifier() {
+        return new DefaultJWTClaimsVerifier<>(null, REQUIRED_CLAIMS);
+    }
+
+    @Bean
+    public StandardJwtAuthenticationConverter jwtAuthenticationConverter() {
+        return new StandardJwtAuthenticationConverter(authorizer, idpUserGroupService, niFiProperties);
+    }
+
+    @Bean
+    public BearerTokenProvider bearerTokenProvider() {
+        return new StandardBearerTokenProvider(jwsSignerProvider());
+    }
+
+    @Bean
+    public StandardJwsSignerProvider jwsSignerProvider() {
+        return new StandardJwsSignerProvider(verificationKeySelector());
+    }
+
+    @Bean
+    public StandardVerificationKeySelector verificationKeySelector() {
+        return new StandardVerificationKeySelector(verificationKeyService(), niFiProperties.getSecurityUserJwsKeyRotationPeriod());
+    }
+
+    @Bean
+    public VerificationKeyService verificationKeyService() {
+        final StateManager stateManager = stateManagerProvider.getStateManager(StandardVerificationKeyService.class.getName());
+        return new StandardVerificationKeyService(stateManager);
+    }
+
+    @Bean
+    public KeyGenerationCommand keyGenerationCommand() {
+        final KeyGenerationCommand command = new KeyGenerationCommand(jwsSignerProvider(), verificationKeySelector());
+        commandScheduler().scheduleAtFixedRate(command, niFiProperties.getSecurityUserJwsKeyRotationPeriod());

Review comment:
       I noticed that if you specify an invalid key rotation period like `PT0M`, the resulting error message could be more friendly:
   ```
   Caused by: org.springframework.beans.BeanInstantiationException: Failed to instantiate [org.apache.nifi.web.security.jwt.key.command.KeyGenerationCommand]: Factory method 'keyGenerationCommand' threw exception; nested exception is java.lang.IllegalArgumentException
   	at org.springframework.beans.factory.support.SimpleInstantiationStrategy.instantiate(SimpleInstantiationStrategy.java:185)
   	at org.springframework.beans.factory.support.ConstructorResolver.instantiate(ConstructorResolver.java:653)
   	... 53 common frames omitted
   Caused by: java.lang.IllegalArgumentException: null
   	at java.util.concurrent.ScheduledThreadPoolExecutor.scheduleAtFixedRate(ScheduledThreadPoolExecutor.java:565)
   	at org.springframework.scheduling.concurrent.ThreadPoolTaskScheduler.scheduleAtFixedRate(ThreadPoolTaskScheduler.java:407)
   ```
   Perhaps `NiFiProperties#getSecurityUserJwsKeyRotationPeriod` could validate the value and include the property name in the error message if it's invalid?

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/key/service/StandardVerificationKeyService.java
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.security.jwt.key.service;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.components.state.StateMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.security.Key;
+import java.security.KeyFactory;
+import java.security.NoSuchAlgorithmException;
+import java.security.spec.InvalidKeySpecException;
+import java.security.spec.KeySpec;
+import java.security.spec.X509EncodedKeySpec;
+import java.time.Instant;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/**
+ * Standard Verification Key Service implemented using State Manager
+ */
+public class StandardVerificationKeyService implements VerificationKeyService {
+    private static final Logger LOGGER = LoggerFactory.getLogger(StandardVerificationKeyService.class);
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper().registerModule(new JavaTimeModule());
+
+    private static final Scope SCOPE = Scope.LOCAL;
+
+    private final StateManager stateManager;
+
+    public StandardVerificationKeyService(final StateManager stateManager) {
+        this.stateManager = stateManager;
+    }
+
+    /**
+     * Find Key using specified Key Identifier
+     *
+     * @param id Key Identifier
+     * @return Optional Key
+     */
+    @Override
+    public Optional<Key> findById(final String id) {
+        final Optional<String> serializedKey = findSerializedKey(id);
+        return serializedKey.map(this::getVerificationKey).map(this::getKey);
+    }
+
+    /**
+     * Delete Expired Verification Keys is synchronized to avoid losing updates from other methods
+     */
+    @Override
+    public synchronized void deleteExpired() {
+        final Map<String, String> state = getStateMap().toMap();
+
+        final Instant now = Instant.now();
+        final Map<String, String> updatedState = state
+                .values()
+                .stream()
+                .map(this::getVerificationKey)
+                .filter(verificationKey -> verificationKey.getExpiration().isAfter(now))
+                .collect(Collectors.toMap(VerificationKey::getId, this::serializeVerificationKey));
+
+        if (updatedState.equals(state)) {
+            LOGGER.debug("Expired Verification Keys not found");
+        } else {
+            try {
+                stateManager.setState(updatedState, SCOPE);
+            } catch (final IOException e) {
+                throw new UncheckedIOException("Delete Expired Verification Keys Failed", e);
+            }
+            LOGGER.debug("Delete Expired Verification Keys: Before [{}] After [{}]", state.size(), updatedState.size());

Review comment:
       This was slightly cryptic to me when I saw it in the logs.  What do you think about:
   ```suggestion
               LOGGER.debug("Delete Expired Verification Keys: Before [{}] keys, After [{}] keys", state.size(), updatedState.size());
   ```

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/jws/JwsSignerProvider.java
##########
@@ -14,31 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.admin.service.action;
+package org.apache.nifi.web.security.jwt.jws;
 
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.KeyDAO;
+import java.time.Instant;
 
 /**
- *
+ * JSON Web Signature Signer Provider supports accessing signer and identifier properties
  */
-public class DeleteKeyAction implements AdministrationAction<Integer> {
-
-    private final Integer keyId;
-
+public interface JwsSignerProvider {
     /**
-     * Creates a new transactions for deleting keys for a specified user based on their keyId.
+     * Get JSON Web Signature Signer Container
      *
-     * @param keyId user identity
+     * @param expiration JSON Web Token Expiration

Review comment:
       It's not clear from the method name or description here that the expiration argument is intended to set the expiration.  Perhaps:
   ```suggestion
        * @param expiration JSON Web Token Expiration to set
   ```

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/revocation/StandardJwtRevocationService.java
##########
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.security.jwt.revocation;
+
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.components.state.StateMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.time.Instant;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Standard JSON Web Token Revocation Service using State Manager
+ */
+public class StandardJwtRevocationService implements JwtRevocationService {
+    private static final Logger LOGGER = LoggerFactory.getLogger(StandardJwtRevocationService.class);
+
+    private static final Scope SCOPE = Scope.LOCAL;
+
+    private final StateManager stateManager;
+
+    public StandardJwtRevocationService(final StateManager stateManager) {
+        this.stateManager = stateManager;
+    }
+
+    /**
+     * Delete Expired Revocations is synchronized is avoid losing updates from setRevoked()
+     */
+    @Override
+    public synchronized void deleteExpired() {
+        final Map<String, String> state = getStateMap().toMap();
+
+        final Instant now = Instant.now();
+        final Map<String, String> updatedState = state
+                .entrySet()
+                .stream()
+                .filter(entry -> Instant.parse(entry.getValue()).isAfter(now))
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (updatedState.equals(state)) {
+            LOGGER.debug("Expired Revocations not found");
+        } else {
+            try {
+                stateManager.setState(updatedState, SCOPE);
+            } catch (final IOException e) {
+                throw new UncheckedIOException("Delete Expired Revocations Failed", e);
+            }
+            LOGGER.debug("Delete Expired Revocations: Before [{}] After [{}]", state.size(), updatedState.size());

Review comment:
       I'm wondering if this and the "Delete Expired Verification Keys: Before..." message should be a higher log level than most of the others in this PR.  I could see a user wanting to see when revocations and verification keys were actually deleted, but not wanting to see every time a verification key was found or an expired revocation was not found.  What do you think?

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/configuration/JwtAuthenticationSecurityConfiguration.java
##########
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.security.configuration;
+
+import com.nimbusds.jose.proc.JWSKeySelector;
+import com.nimbusds.jose.proc.SecurityContext;
+import com.nimbusds.jwt.proc.DefaultJWTClaimsVerifier;
+import com.nimbusds.jwt.proc.DefaultJWTProcessor;
+import com.nimbusds.jwt.proc.JWTClaimsSetVerifier;
+import com.nimbusds.jwt.proc.JWTProcessor;
+import org.apache.nifi.admin.service.IdpUserGroupService;
+import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.components.state.StateManagerProvider;
+import org.apache.nifi.util.NiFiProperties;
+import org.apache.nifi.web.security.jwt.converter.StandardJwtAuthenticationConverter;
+import org.apache.nifi.web.security.jwt.jws.StandardJWSKeySelector;
+import org.apache.nifi.web.security.jwt.jws.StandardJwsSignerProvider;
+import org.apache.nifi.web.security.jwt.key.command.KeyExpirationCommand;
+import org.apache.nifi.web.security.jwt.key.command.KeyGenerationCommand;
+import org.apache.nifi.web.security.jwt.key.StandardVerificationKeySelector;
+import org.apache.nifi.web.security.jwt.key.service.StandardVerificationKeyService;
+import org.apache.nifi.web.security.jwt.key.service.VerificationKeyService;
+import org.apache.nifi.web.security.jwt.provider.BearerTokenProvider;
+import org.apache.nifi.web.security.jwt.provider.StandardBearerTokenProvider;
+import org.apache.nifi.web.security.jwt.revocation.JwtLogoutListener;
+import org.apache.nifi.web.security.jwt.revocation.JwtRevocationService;
+import org.apache.nifi.web.security.jwt.revocation.JwtRevocationValidator;
+import org.apache.nifi.web.security.jwt.revocation.StandardJwtLogoutListener;
+import org.apache.nifi.web.security.jwt.revocation.StandardJwtRevocationService;
+import org.apache.nifi.web.security.jwt.revocation.command.RevocationExpirationCommand;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.scheduling.concurrent.ThreadPoolTaskScheduler;
+import org.springframework.security.oauth2.core.DelegatingOAuth2TokenValidator;
+import org.springframework.security.oauth2.core.OAuth2TokenValidator;
+import org.springframework.security.oauth2.jwt.Jwt;
+import org.springframework.security.oauth2.jwt.JwtDecoder;
+import org.springframework.security.oauth2.jwt.JwtValidators;
+import org.springframework.security.oauth2.jwt.NimbusJwtDecoder;
+import org.springframework.security.oauth2.server.resource.authentication.JwtAuthenticationProvider;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * JSON Web Token Configuration for Authentication Security
+ */
+@Configuration
+public class JwtAuthenticationSecurityConfiguration {
+    private static final Set<String> REQUIRED_CLAIMS = new HashSet<>(Arrays.asList("sub", "iss", "aud", "nbf", "iat", "exp", "jti"));
+
+    private final NiFiProperties niFiProperties;
+
+    private final Authorizer authorizer;
+
+    private final IdpUserGroupService idpUserGroupService;
+
+    private final StateManagerProvider stateManagerProvider;
+
+    @Autowired
+    public JwtAuthenticationSecurityConfiguration(
+            final NiFiProperties niFiProperties,
+            final Authorizer authorizer,
+            final IdpUserGroupService idpUserGroupService,
+            final StateManagerProvider stateManagerProvider
+    ) {
+        this.niFiProperties = niFiProperties;
+        this.authorizer = authorizer;
+        this.idpUserGroupService = idpUserGroupService;
+        this.stateManagerProvider = stateManagerProvider;
+    }
+
+    @Bean
+    public JwtAuthenticationProvider jwtAuthenticationProvider() {
+        final JwtAuthenticationProvider jwtAuthenticationProvider = new JwtAuthenticationProvider(jwtDecoder());
+        jwtAuthenticationProvider.setJwtAuthenticationConverter(jwtAuthenticationConverter());
+        return jwtAuthenticationProvider;
+    }
+
+    @Bean
+    public JwtDecoder jwtDecoder() {
+        final NimbusJwtDecoder jwtDecoder = new NimbusJwtDecoder(jwtProcessor());
+        final OAuth2TokenValidator<Jwt> jwtValidator = new DelegatingOAuth2TokenValidator<>(
+                JwtValidators.createDefault(),
+                jwtRevocationValidator()
+        );
+        jwtDecoder.setJwtValidator(jwtValidator);
+        return jwtDecoder;
+    }
+
+    @Bean
+    public OAuth2TokenValidator<Jwt> jwtRevocationValidator() {
+        return new JwtRevocationValidator(jwtRevocationService());
+    }
+
+    @Bean
+    public JwtRevocationService jwtRevocationService() {
+        final StateManager stateManager = stateManagerProvider.getStateManager(StandardJwtRevocationService.class.getName());
+        return new StandardJwtRevocationService(stateManager);
+    }
+
+    @Bean
+    public JwtLogoutListener jwtLogoutListener() {
+        return new StandardJwtLogoutListener(jwtDecoder(), jwtRevocationService());
+    }
+
+    @Bean
+    public JWTProcessor<SecurityContext> jwtProcessor() {
+        final DefaultJWTProcessor<SecurityContext> jwtProcessor = new DefaultJWTProcessor<>();
+        jwtProcessor.setJWSKeySelector(jwsKeySelector());
+        jwtProcessor.setJWTClaimsSetVerifier(claimsSetVerifier());
+        return jwtProcessor;
+    }
+
+    @Bean
+    public JWSKeySelector<SecurityContext> jwsKeySelector() {
+        return new StandardJWSKeySelector<>(verificationKeySelector());
+    }
+
+    @Bean
+    public JWTClaimsSetVerifier<SecurityContext> claimsSetVerifier() {
+        return new DefaultJWTClaimsVerifier<>(null, REQUIRED_CLAIMS);
+    }
+
+    @Bean
+    public StandardJwtAuthenticationConverter jwtAuthenticationConverter() {
+        return new StandardJwtAuthenticationConverter(authorizer, idpUserGroupService, niFiProperties);
+    }
+
+    @Bean
+    public BearerTokenProvider bearerTokenProvider() {
+        return new StandardBearerTokenProvider(jwsSignerProvider());
+    }
+
+    @Bean
+    public StandardJwsSignerProvider jwsSignerProvider() {
+        return new StandardJwsSignerProvider(verificationKeySelector());
+    }
+
+    @Bean
+    public StandardVerificationKeySelector verificationKeySelector() {
+        return new StandardVerificationKeySelector(verificationKeyService(), niFiProperties.getSecurityUserJwsKeyRotationPeriod());
+    }
+
+    @Bean
+    public VerificationKeyService verificationKeyService() {
+        final StateManager stateManager = stateManagerProvider.getStateManager(StandardVerificationKeyService.class.getName());
+        return new StandardVerificationKeyService(stateManager);
+    }
+
+    @Bean
+    public KeyGenerationCommand keyGenerationCommand() {
+        final KeyGenerationCommand command = new KeyGenerationCommand(jwsSignerProvider(), verificationKeySelector());
+        commandScheduler().scheduleAtFixedRate(command, niFiProperties.getSecurityUserJwsKeyRotationPeriod());
+        return command;
+    }
+
+    @Bean
+    public KeyExpirationCommand keyExpirationCommand() {
+        final KeyExpirationCommand command = new KeyExpirationCommand(verificationKeyService());
+        commandScheduler().scheduleAtFixedRate(command, niFiProperties.getSecurityUserJwsKeyRotationPeriod());
+        return command;
+    }
+
+    @Bean
+    public RevocationExpirationCommand revocationExpirationCommand() {
+        final RevocationExpirationCommand command = new RevocationExpirationCommand(jwtRevocationService());
+        commandScheduler().scheduleAtFixedRate(command, niFiProperties.getSecurityUserJwsKeyRotationPeriod());

Review comment:
       In the unlikely scenario that the user configures the key rotation period to be longer than the login identity provider's authentication expiration, this wouldn't expire the revocation in time, right?  Perhaps that could be considered a user error, though.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/configuration/JwtAuthenticationSecurityConfiguration.java
##########
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.security.configuration;
+
+import com.nimbusds.jose.proc.JWSKeySelector;
+import com.nimbusds.jose.proc.SecurityContext;
+import com.nimbusds.jwt.proc.DefaultJWTClaimsVerifier;
+import com.nimbusds.jwt.proc.DefaultJWTProcessor;
+import com.nimbusds.jwt.proc.JWTClaimsSetVerifier;
+import com.nimbusds.jwt.proc.JWTProcessor;
+import org.apache.nifi.admin.service.IdpUserGroupService;
+import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.components.state.StateManagerProvider;
+import org.apache.nifi.util.NiFiProperties;
+import org.apache.nifi.web.security.jwt.converter.StandardJwtAuthenticationConverter;
+import org.apache.nifi.web.security.jwt.jws.StandardJWSKeySelector;
+import org.apache.nifi.web.security.jwt.jws.StandardJwsSignerProvider;
+import org.apache.nifi.web.security.jwt.key.command.KeyExpirationCommand;
+import org.apache.nifi.web.security.jwt.key.command.KeyGenerationCommand;
+import org.apache.nifi.web.security.jwt.key.StandardVerificationKeySelector;
+import org.apache.nifi.web.security.jwt.key.service.StandardVerificationKeyService;
+import org.apache.nifi.web.security.jwt.key.service.VerificationKeyService;
+import org.apache.nifi.web.security.jwt.provider.BearerTokenProvider;
+import org.apache.nifi.web.security.jwt.provider.StandardBearerTokenProvider;
+import org.apache.nifi.web.security.jwt.revocation.JwtLogoutListener;
+import org.apache.nifi.web.security.jwt.revocation.JwtRevocationService;
+import org.apache.nifi.web.security.jwt.revocation.JwtRevocationValidator;
+import org.apache.nifi.web.security.jwt.revocation.StandardJwtLogoutListener;
+import org.apache.nifi.web.security.jwt.revocation.StandardJwtRevocationService;
+import org.apache.nifi.web.security.jwt.revocation.command.RevocationExpirationCommand;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.scheduling.concurrent.ThreadPoolTaskScheduler;
+import org.springframework.security.oauth2.core.DelegatingOAuth2TokenValidator;
+import org.springframework.security.oauth2.core.OAuth2TokenValidator;
+import org.springframework.security.oauth2.jwt.Jwt;
+import org.springframework.security.oauth2.jwt.JwtDecoder;
+import org.springframework.security.oauth2.jwt.JwtValidators;
+import org.springframework.security.oauth2.jwt.NimbusJwtDecoder;
+import org.springframework.security.oauth2.server.resource.authentication.JwtAuthenticationProvider;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * JSON Web Token Configuration for Authentication Security
+ */
+@Configuration
+public class JwtAuthenticationSecurityConfiguration {
+    private static final Set<String> REQUIRED_CLAIMS = new HashSet<>(Arrays.asList("sub", "iss", "aud", "nbf", "iat", "exp", "jti"));

Review comment:
       Although developers familiar with JWT will likely know what these abbreviations are, what do you think about adding some private constants for others?  Or at least a comment including their meanings.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org