From 1e917906ab2abd87dbf5ae9aaec781538713cf27 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 4 Jun 2025 06:01:05 -0700 Subject: [PATCH] KAFKA-18573: Add support for OAuth jwt-bearer grant type (#19754) Adding support for the `urn:ietf:params:oauth:grant-type:jwt-bearer` grant type (AKA `jwt-bearer`). Includes further refactoring of the existing OAuth layer and addition of generic JWT assertion layer that can be leveraged in the future. This constitutes the main piece of the JWT Bearer grant type support. Forthcoming commits/PRs will include improvements for both the `client_credentials` and `jwt-bearer` grant types in the following areas: * Integration test coverage (KAFKA-19153) * Unit test coverage (KAFKA-19308) * Top-level documentation (KAFKA-19152) * Improvements to and documentation for `OAuthCompatibilityTool` (KAFKA-19307) Reviewers: Manikumar Reddy , Lianet Magrans --------- Co-authored-by: Zachary Hamilton <77027819+zacharydhamilton@users.noreply.github.com> Co-authored-by: Lianet Magrans <98415067+lianetm@users.noreply.github.com> --- checkstyle/import-control.xml | 1 + .../kafka/common/config/SaslConfigs.java | 188 ++++++++++++- .../internals/BrokerSecurityConfigs.java | 20 ++ .../secured => }/BrokerJwtValidator.java | 116 ++++---- .../ClientCredentialsJwtRetriever.java | 249 ++++++++++++++++++ .../secured => }/ClientJwtValidator.java | 44 ++-- .../oauthbearer/DefaultJwtRetriever.java | 90 +++++++ .../oauthbearer/DefaultJwtValidator.java | 79 ++++++ .../oauthbearer/FileJwtRetriever.java | 59 +++++ .../oauthbearer/JwtBearerJwtRetriever.java | 181 +++++++++++++ .../{internals/secured => }/JwtRetriever.java | 26 +- .../oauthbearer/JwtRetrieverException.java | 40 +++ .../{internals/secured => }/JwtValidator.java | 19 +- ...eption.java => JwtValidatorException.java} | 21 +- .../OAuthBearerLoginCallbackHandler.java | 56 ++-- .../OAuthBearerValidatorCallbackHandler.java | 143 ++-------- .../internals/secured/CachedFile.java | 179 +++++++++++++ .../secured/ClaimValidationUtils.java | 40 +-- .../ClientCredentialsRequestFormatter.java | 91 +++++++ .../CloseableVerificationKeyResolver.java | 21 +- .../internals/secured/ConfigurationUtils.java | 211 +++++++++++++-- .../secured/DefaultJwtRetriever.java | 131 --------- .../secured/DefaultJwtValidator.java | 108 -------- .../internals/secured/FileJwtRetriever.java | 57 ---- .../internals/secured/HttpJwtRetriever.java | 158 +++-------- ...nitable.java => HttpRequestFormatter.java} | 16 +- .../internals/secured/JaasOptionsUtils.java | 39 +-- .../JwksFileVerificationKeyResolver.java | 63 +++-- .../secured/JwtBearerRequestFormatter.java | 61 +++++ .../internals/secured/JwtResponseParser.java | 65 +++++ .../secured/OAuthBearerConfigurable.java | 88 +++++++ .../secured/RefreshingHttpsJwks.java | 7 +- ...shingHttpsJwksVerificationKeyResolver.java | 18 +- .../internals/secured/SerializedJwt.java | 10 +- .../VerificationKeyResolverFactory.java | 143 ++++++++-- .../secured/assertion/AssertionCreator.java | 96 +++++++ .../assertion/AssertionJwtTemplate.java | 73 +++++ .../secured/assertion/AssertionUtils.java | 150 +++++++++++ .../assertion/DefaultAssertionCreator.java | 96 +++++++ .../DynamicAssertionJwtTemplate.java | 81 ++++++ .../assertion/FileAssertionCreator.java | 44 ++++ .../assertion/FileAssertionJwtTemplate.java | 165 ++++++++++++ .../LayeredAssertionJwtTemplate.java | 108 ++++++++ .../assertion/StaticAssertionJwtTemplate.java | 52 ++++ .../secured => }/BrokerJwtValidatorTest.java | 20 +- .../ClaimValidationUtilsTest.java | 33 +-- .../secured => }/ClientJwtValidatorTest.java | 6 +- .../secured => }/DefaultJwtRetrieverTest.java | 99 +++---- .../secured => }/DefaultJwtValidatorTest.java | 22 +- .../JwtBearerJwtRetrieverTest.java | 152 +++++++++++ .../secured => }/JwtValidatorTest.java | 19 +- .../OAuthBearerLoginCallbackHandlerTest.java | 77 +++--- ...uthBearerValidatorCallbackHandlerTest.java | 67 +++-- .../internals/secured/AccessTokenBuilder.java | 8 +- .../internals/secured/CachedFileTest.java | 151 +++++++++++ ...ClientCredentialsRequestFormatterTest.java | 141 ++++++++++ .../secured/ConfigurationUtilsTest.java | 93 ++++--- .../secured/HttpJwtRetrieverTest.java | 103 -------- .../secured/JwtResponseParserTest.java | 71 +++++ .../internals/secured/OAuthBearerTest.java | 133 +++++++--- .../VerificationKeyResolverFactoryTest.java | 36 +-- .../DefaultAssertionCreatorTest.java | 193 ++++++++++++++ .../DynamicAssertionJwtTemplateTest.java | 83 ++++++ .../assertion/FileAssertionCreatorTest.java | 51 ++++ .../FileAssertionJwtTemplateTest.java | 129 +++++++++ .../unit/kafka/server/KafkaConfigTest.scala | 31 ++- .../kafka/tools/OAuthCompatibilityTool.java | 65 +++-- 67 files changed, 4230 insertions(+), 1256 deletions(-) rename clients/src/main/java/org/apache/kafka/common/security/oauthbearer/{internals/secured => }/BrokerJwtValidator.java (56%) create mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/ClientCredentialsJwtRetriever.java rename clients/src/main/java/org/apache/kafka/common/security/oauthbearer/{internals/secured => }/ClientJwtValidator.java (76%) create mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtRetriever.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtValidator.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/FileJwtRetriever.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtBearerJwtRetriever.java rename clients/src/main/java/org/apache/kafka/common/security/oauthbearer/{internals/secured => }/JwtRetriever.java (73%) create mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtRetrieverException.java rename clients/src/main/java/org/apache/kafka/common/security/oauthbearer/{internals/secured => }/JwtValidator.java (81%) rename clients/src/main/java/org/apache/kafka/common/security/oauthbearer/{internals/secured/ValidateException.java => JwtValidatorException.java} (62%) create mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CachedFile.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientCredentialsRequestFormatter.java delete mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetriever.java delete mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java delete mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileJwtRetriever.java rename clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/{Initable.java => HttpRequestFormatter.java} (66%) create mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtBearerRequestFormatter.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtResponseParser.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerConfigurable.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/AssertionCreator.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/AssertionJwtTemplate.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/AssertionUtils.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/DefaultAssertionCreator.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/DynamicAssertionJwtTemplate.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/FileAssertionCreator.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/FileAssertionJwtTemplate.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/LayeredAssertionJwtTemplate.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/StaticAssertionJwtTemplate.java rename clients/src/test/java/org/apache/kafka/common/security/oauthbearer/{internals/secured => }/BrokerJwtValidatorTest.java (79%) rename clients/src/test/java/org/apache/kafka/common/security/oauthbearer/{internals/secured => }/ClaimValidationUtilsTest.java (72%) rename clients/src/test/java/org/apache/kafka/common/security/oauthbearer/{internals/secured => }/ClientJwtValidatorTest.java (83%) rename clients/src/test/java/org/apache/kafka/common/security/oauthbearer/{internals/secured => }/DefaultJwtRetrieverTest.java (60%) rename clients/src/test/java/org/apache/kafka/common/security/oauthbearer/{internals/secured => }/DefaultJwtValidatorTest.java (74%) create mode 100644 clients/src/test/java/org/apache/kafka/common/security/oauthbearer/JwtBearerJwtRetrieverTest.java rename clients/src/test/java/org/apache/kafka/common/security/oauthbearer/{internals/secured => }/JwtValidatorTest.java (73%) create mode 100644 clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CachedFileTest.java create mode 100644 clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientCredentialsRequestFormatterTest.java create mode 100644 clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtResponseParserTest.java create mode 100644 clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/DefaultAssertionCreatorTest.java create mode 100644 clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/DynamicAssertionJwtTemplateTest.java create mode 100644 clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/FileAssertionCreatorTest.java create mode 100644 clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/FileAssertionJwtTemplateTest.java diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index b130609bd31..9fc1bcd7eff 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -145,6 +145,7 @@ + diff --git a/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java b/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java index 15e23270d6f..b78d96eaac8 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java +++ b/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.common.config; +import org.apache.kafka.common.config.ConfigDef.CaseInsensitiveValidString; import org.apache.kafka.common.config.ConfigDef.Range; public class SaslConfigs { @@ -129,6 +130,173 @@ public class SaslConfigs { + " authentication provider." + LOGIN_EXPONENTIAL_BACKOFF_NOTE; + public static final String SASL_OAUTHBEARER_JWT_RETRIEVER_CLASS = "sasl.oauthbearer.jwt.retriever.class"; + public static final String DEFAULT_SASL_OAUTHBEARER_JWT_RETRIEVER_CLASS = "org.apache.kafka.common.security.oauthbearer.DefaultJwtRetriever"; + public static final String SASL_OAUTHBEARER_JWT_RETRIEVER_CLASS_DOC = "

The fully-qualified class name of a JwtRetriever implementation used to" + + " request tokens from the identity provider.

" + + "

The default configuration value represents a class that maintains backward compatibility with previous versions of" + + " Apache Kafka. The default implementation uses the configuration to determine which concrete implementation to create." + + "

Other implementations that are provided include:

" + + "
    " + + "
  • org.apache.kafka.common.security.oauthbearer.ClientCredentialsJwtRetriever
  • " + + "
  • org.apache.kafka.common.security.oauthbearer.DefaultJwtRetriever
  • " + + "
  • org.apache.kafka.common.security.oauthbearer.FileJwtRetriever
  • " + + "
  • org.apache.kafka.common.security.oauthbearer.JwtBearerJwtRetriever
  • " + + "
"; + + public static final String SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS = "sasl.oauthbearer.jwt.validator.class"; + public static final String DEFAULT_SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS = "org.apache.kafka.common.security.oauthbearer.DefaultJwtValidator"; + public static final String SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS_DOC = "

The fully-qualified class name of a JwtValidator implementation used to" + + " validate the JWT from the identity provider.

" + + "

The default validator (org.apache.kafka.common.security.oauthbearer.DefaultJwtValidator) maintains backward compatibility with previous" + + " versions of Apache Kafka. The default validator uses configuration to determine which concrete implementation to create." + + "

The built-in JwtValidator implementations are:

" + + "
    " + + "
  • org.apache.kafka.common.security.oauthbearer.BrokerJwtValidator
  • " + + "
  • org.apache.kafka.common.security.oauthbearer.ClientJwtValidator
  • " + + "
  • org.apache.kafka.common.security.oauthbearer.DefaultJwtValidator
  • " + + "
"; + + public static final String SASL_OAUTHBEARER_SCOPE = "sasl.oauthbearer.scope"; + public static final String SASL_OAUTHBEARER_SCOPE_DOC = "

This is the level of access a client application is granted to a resource or API which is" + + " included in the token request. If provided, it should match one or more scopes configured in the identity provider.

" + + "

" + + "The scope was previously stored as part of the sasl.jaas.config configuration with the key scope." + + " For backward compatibility, the scope JAAS option can still be used, but it is deprecated and will be removed in a future version." + + "

" + + "

Order of precedence:

" + + "
    " + + "
  • sasl.oauthbearer.scope from configuration
  • " + + "
  • scope from JAAS
  • " + + "
"; + + public static final String SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_ID = "sasl.oauthbearer.client.credentials.client.id"; + public static final String SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_ID_DOC = "

The ID (defined in/by the OAuth identity provider) to identify the client" + + " requesting the token.

" + + "

" + + "The client ID was previously stored as part of the sasl.jaas.config configuration with the key clientId." + + " For backward compatibility, the clientId JAAS option can still be used, but it is deprecated and will be removed in a future version." + + "

" + + "

Order of precedence:

" + + "
    " + + "
  • sasl.oauthbearer.client.credentials.client.id from configuration
  • " + + "
  • clientId from JAAS
  • " + + "
"; + + public static final String SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_SECRET = "sasl.oauthbearer.client.credentials.client.secret"; + public static final String SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_SECRET_DOC = "

The secret (defined by either the user or preassigned, depending on the" + + " identity provider) of the client requesting the token.

" + + "

" + + "The client secret was previously stored as part of the sasl.jaas.config configuration with the key clientSecret." + + " For backward compatibility, the clientSecret JAAS option can still be used, but it is deprecated and will be removed in a future version." + + "

" + + "

Order of precedence:

" + + "
    " + + "
  • sasl.oauthbearer.client.credentials.client.secret from configuration
  • " + + "
  • clientSecret from JAAS
  • " + + "
"; + + private static final String ASSERTION_FILE_MUTUAL_EXCLUSION_NOTICE = "

Note: If a value for sasl.oauthbearer.assertion.file is provided," + + " this configuration will be ignored.

"; + + public static final String SASL_OAUTHBEARER_ASSERTION_ALGORITHM = "sasl.oauthbearer.assertion.algorithm"; + public static final String DEFAULT_SASL_OAUTHBEARER_ASSERTION_ALGORITHM = "RS256"; + public static final String SASL_OAUTHBEARER_ASSERTION_ALGORITHM_DOC = "

The algorithm the Apache Kafka client should use to sign the assertion sent" + + " to the identity provider. It is also used as the value of the OAuth alg (Algorithm) header in the JWT assertion.

" + + ASSERTION_FILE_MUTUAL_EXCLUSION_NOTICE; + + public static final String SASL_OAUTHBEARER_ASSERTION_CLAIM_AUD = "sasl.oauthbearer.assertion.claim.aud"; + public static final String SASL_OAUTHBEARER_ASSERTION_CLAIM_AUD_DOC = "

The JWT aud (Audience) claim which will be included in the " + + " client JWT assertion created locally.

" + + ASSERTION_FILE_MUTUAL_EXCLUSION_NOTICE; + + public static final String SASL_OAUTHBEARER_ASSERTION_CLAIM_EXP_SECONDS = "sasl.oauthbearer.assertion.claim.exp.seconds"; + public static final int DEFAULT_SASL_OAUTHBEARER_ASSERTION_CLAIM_EXP_SECONDS = 300; + public static final String SASL_OAUTHBEARER_ASSERTION_CLAIM_EXP_SECONDS_DOC = "

The number of seconds in the future for which the JWT is valid." + + " The value is used to determine the JWT exp (Expiration) claim based on the current system time when the JWT is created.

" + + "

The formula to generate the exp claim is very simple:

" + + "
"
+        + "Let:\n\n"
+        + "  x = the current timestamp in seconds, on client\n"
+        + "  y = the value of this configuration\n"
+        + "\n"
+        + "Then:\n\n"
+        + "  exp = x + y\n"
+        + "
" + + ASSERTION_FILE_MUTUAL_EXCLUSION_NOTICE; + + public static final String SASL_OAUTHBEARER_ASSERTION_CLAIM_ISS = "sasl.oauthbearer.assertion.claim.iss"; + public static final String SASL_OAUTHBEARER_ASSERTION_CLAIM_ISS_DOC = "

The value to be used as the iss (Issuer) claim which will be included in the" + + " client JWT assertion created locally.

" + + ASSERTION_FILE_MUTUAL_EXCLUSION_NOTICE; + + public static final String SASL_OAUTHBEARER_ASSERTION_CLAIM_JTI_INCLUDE = "sasl.oauthbearer.assertion.claim.jti.include"; + public static final boolean DEFAULT_SASL_OAUTHBEARER_ASSERTION_CLAIM_JTI_INCLUDE = false; + public static final String SASL_OAUTHBEARER_ASSERTION_CLAIM_JTI_INCLUDE_DOC = "

Flag that determines if the JWT assertion should generate a unique ID for the" + + " JWT and include it in the jti (JWT ID) claim.

" + + ASSERTION_FILE_MUTUAL_EXCLUSION_NOTICE; + + public static final String SASL_OAUTHBEARER_ASSERTION_CLAIM_NBF_SECONDS = "sasl.oauthbearer.assertion.claim.nbf.seconds"; + public static final int DEFAULT_SASL_OAUTHBEARER_ASSERTION_CLAIM_NBF_SECONDS = 60; + public static final String SASL_OAUTHBEARER_ASSERTION_CLAIM_NBF_SECONDS_DOC = "

The number of seconds in the past from which the JWT is valid." + + " The value is used to determine the JWT nbf (Not Before) claim based on the current system time when the JWT is created.

" + + "

The formula to generate the nbf claim is very simple:

" + + "
"
+        + "Let:\n\n"
+        + "  x = the current timestamp in seconds, on client\n"
+        + "  y = the value of this configuration\n"
+        + "\n"
+        + "Then:\n\n"
+        + "  nbf = x - y\n"
+        + "
" + + ASSERTION_FILE_MUTUAL_EXCLUSION_NOTICE; + + public static final String SASL_OAUTHBEARER_ASSERTION_CLAIM_SUB = "sasl.oauthbearer.assertion.claim.sub"; + public static final String SASL_OAUTHBEARER_ASSERTION_CLAIM_SUB_DOC = "

The value to be used as the sub (Subject) claim which will be included in the" + + " client JWT assertion created locally.

" + + ASSERTION_FILE_MUTUAL_EXCLUSION_NOTICE; + + public static final String SASL_OAUTHBEARER_ASSERTION_FILE = "sasl.oauthbearer.assertion.file"; + public static final String SASL_OAUTHBEARER_ASSERTION_FILE_DOC = "

File that contains a pre-generated JWT assertion.

" + + "

The underlying implementation caches the file contents to avoid the performance hit of loading the file on each access. The caching mechanism will detect when" + + "the file changes to allow for the file to be reloaded on modifications. This allows for "live" assertion rotation without restarting the Kafka client.

" + + "

The file contains the assertion in the serialized, three part JWT format:

" + + "
    " + + "
  1. The header section is a base 64-encoded JWT header that contains values like alg (Algorithm)," + + " typ (Type, always the literal value JWT), etc.
  2. " + + "
  3. The payload section includes the base 64-encoded set of JWT claims, such as aud (Audience), iss (Issuer)," + + " sub (Subject), etc.
  4. " + + "
  5. The signature section is the concatenated header and payload sections that was signed using a private key
  6. " + + "
" + + "

See RFC 7519 and RFC 7515" + + " for more details on the JWT and JWS formats.

" + + "

Note: If a value for sasl.oauthbearer.assertion.file is provided, all other" + + " sasl.oauthbearer.assertion.* configurations are ignored.

"; + + public static final String SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_FILE = "sasl.oauthbearer.assertion.private.key.file"; + public static final String SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_FILE_DOC = "

File that contains a private key in the standard PEM format which is used to" + + " sign the JWT assertion sent to the identity provider.

" + + "

The underlying implementation caches the file contents to avoid the performance hit of loading the file on each access. The caching mechanism will detect when" + + " the file changes to allow for the file to be reloaded on modifications. This allows for "live" private key rotation without restarting the Kafka client.

" + + ASSERTION_FILE_MUTUAL_EXCLUSION_NOTICE; + + public static final String SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_PASSPHRASE = "sasl.oauthbearer.assertion.private.key.passphrase"; + public static final String SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_PASSPHRASE_DOC = "

The optional passphrase to decrypt the private key file specified by" + + " sasl.oauthbearer.assertion.private.key.file.

" + + "

Note: If the file referred to by sasl.oauthbearer.assertion.private.key.file is modified on the file system at runtime and it was" + + " created with a different passphrase than it was previously, the client will not be able to access the private key file because the passphrase is now" + + " out of date. For that reason, when using private key passphrases, either use the same passphrase each time, or—for improved security—restart" + + " the Kafka client using the new passphrase configuration.

" + + ASSERTION_FILE_MUTUAL_EXCLUSION_NOTICE; + + public static final String SASL_OAUTHBEARER_ASSERTION_TEMPLATE_FILE = "sasl.oauthbearer.assertion.template.file"; + public static final String SASL_OAUTHBEARER_ASSERTION_TEMPLATE_FILE_DOC = "

This optional configuration specifies the file containing the JWT headers and/or" + + " payload claims to be used when creating the JWT assertion.

" + + "

Not all identity providers require the same set of claims; some may require a given claim while others may prohibit it." + + " In order to provide the most flexibility, this configuration allows the user to provide the static header values and claims" + + " that are to be included in the JWT.

" + + ASSERTION_FILE_MUTUAL_EXCLUSION_NOTICE; + public static final String SASL_OAUTHBEARER_SCOPE_CLAIM_NAME = "sasl.oauthbearer.scope.claim.name"; public static final String DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME = "scope"; public static final String SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DOC = "The OAuth claim for the scope is often named \"" + DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME + "\", but this (optional)" @@ -143,8 +311,8 @@ public class SaslConfigs { public static final String SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL = "sasl.oauthbearer.token.endpoint.url"; public static final String SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_DOC = "The URL for the OAuth/OIDC identity provider. If the URL is HTTP(S)-based, it is the issuer's token" - + " endpoint URL to which requests will be made to login based on the configuration in " + SASL_JAAS_CONFIG + ". If the URL is file-based, it" - + " specifies a file containing an access token (in JWT serialized form) issued by the OAuth/OIDC identity provider to use for authorization."; + + " endpoint URL to which requests will be made to login based on the configuration in " + SASL_OAUTHBEARER_JWT_RETRIEVER_CLASS + ". If the URL is" + + " file-based, it specifies a file containing an access token (in JWT serialized form) issued by the OAuth/OIDC identity provider to use for authorization."; public static final String SASL_OAUTHBEARER_JWKS_ENDPOINT_URL = "sasl.oauthbearer.jwks.endpoint.url"; public static final String SASL_OAUTHBEARER_JWKS_ENDPOINT_URL_DOC = "The OAuth/OIDC provider URL from which the provider's" @@ -215,6 +383,22 @@ public class SaslConfigs { .define(SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS, ConfigDef.Type.INT, null, ConfigDef.Importance.LOW, SASL_LOGIN_READ_TIMEOUT_MS_DOC) .define(SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS, ConfigDef.Type.LONG, DEFAULT_SASL_LOGIN_RETRY_BACKOFF_MAX_MS, ConfigDef.Importance.LOW, SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DOC) .define(SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS, ConfigDef.Type.LONG, DEFAULT_SASL_LOGIN_RETRY_BACKOFF_MS, ConfigDef.Importance.LOW, SASL_LOGIN_RETRY_BACKOFF_MS_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_JWT_RETRIEVER_CLASS, ConfigDef.Type.CLASS, DEFAULT_SASL_OAUTHBEARER_JWT_RETRIEVER_CLASS, ConfigDef.Importance.MEDIUM, SASL_OAUTHBEARER_JWT_RETRIEVER_CLASS_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS, ConfigDef.Type.CLASS, DEFAULT_SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS, ConfigDef.Importance.MEDIUM, SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_SCOPE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, SASL_OAUTHBEARER_SCOPE_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_ID, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_ID_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_SECRET, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.MEDIUM, SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_SECRET_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_ALGORITHM, ConfigDef.Type.STRING, DEFAULT_SASL_OAUTHBEARER_ASSERTION_ALGORITHM, CaseInsensitiveValidString.in("ES256", "RS256"), ConfigDef.Importance.MEDIUM, SASL_OAUTHBEARER_ASSERTION_ALGORITHM_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_AUD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, SASL_OAUTHBEARER_ASSERTION_CLAIM_AUD_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_EXP_SECONDS, ConfigDef.Type.INT, DEFAULT_SASL_OAUTHBEARER_ASSERTION_CLAIM_EXP_SECONDS, Range.between(0, 86400), ConfigDef.Importance.LOW, SASL_OAUTHBEARER_ASSERTION_CLAIM_EXP_SECONDS_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_ISS, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, SASL_OAUTHBEARER_ASSERTION_CLAIM_ISS_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_JTI_INCLUDE, ConfigDef.Type.BOOLEAN, DEFAULT_SASL_OAUTHBEARER_ASSERTION_CLAIM_JTI_INCLUDE, ConfigDef.Importance.MEDIUM, SASL_OAUTHBEARER_ASSERTION_CLAIM_JTI_INCLUDE_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_NBF_SECONDS, ConfigDef.Type.INT, DEFAULT_SASL_OAUTHBEARER_ASSERTION_CLAIM_NBF_SECONDS, Range.between(0, 3600), ConfigDef.Importance.LOW, SASL_OAUTHBEARER_ASSERTION_CLAIM_NBF_SECONDS_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_SUB, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, SASL_OAUTHBEARER_ASSERTION_CLAIM_SUB_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_FILE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, SASL_OAUTHBEARER_ASSERTION_FILE_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_FILE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_FILE_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_PASSPHRASE, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.MEDIUM, SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_PASSPHRASE_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_TEMPLATE_FILE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, SASL_OAUTHBEARER_ASSERTION_TEMPLATE_FILE_DOC) .define(SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME, ConfigDef.Type.STRING, DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME, ConfigDef.Importance.LOW, SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DOC) .define(SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME, ConfigDef.Type.STRING, DEFAULT_SASL_OAUTHBEARER_SUB_CLAIM_NAME, ConfigDef.Importance.LOW, SASL_OAUTHBEARER_SUB_CLAIM_NAME_DOC) .define(SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_DOC) diff --git a/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java b/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java index a8947ede154..a3a0baf1e39 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java +++ b/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java @@ -136,6 +136,10 @@ public class BrokerSecurityConfigs { // The allowlist of the SASL OAUTHBEARER endpoints public static final String ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG = "org.apache.kafka.sasl.oauthbearer.allowed.urls"; public static final String ALLOWED_SASL_OAUTHBEARER_URLS_DEFAULT = ""; + + public static final String ALLOWED_SASL_OAUTHBEARER_FILES_CONFIG = "org.apache.kafka.sasl.oauthbearer.allowed.files"; + public static final String ALLOWED_SASL_OAUTHBEARER_FILES_DEFAULT = ""; + public static final ConfigDef CONFIG_DEF = new ConfigDef() // General Security Configuration .define(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG, LONG, BrokerSecurityConfigs.DEFAULT_CONNECTIONS_MAX_REAUTH_MS, MEDIUM, BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_DOC) @@ -190,6 +194,22 @@ public class BrokerSecurityConfigs { .define(SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS, INT, null, LOW, SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS_DOC) .define(SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS, LONG, SaslConfigs.DEFAULT_SASL_LOGIN_RETRY_BACKOFF_MAX_MS, LOW, SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DOC) .define(SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS, LONG, SaslConfigs.DEFAULT_SASL_LOGIN_RETRY_BACKOFF_MS, LOW, SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_JWT_RETRIEVER_CLASS, CLASS, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWT_RETRIEVER_CLASS, MEDIUM, SaslConfigs.SASL_OAUTHBEARER_JWT_RETRIEVER_CLASS_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS, CLASS, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS, MEDIUM, SaslConfigs.SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_SCOPE, STRING, null, MEDIUM, SaslConfigs.SASL_OAUTHBEARER_SCOPE_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_ID, STRING, null, MEDIUM, SaslConfigs.SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_ID_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_SECRET, PASSWORD, null, MEDIUM, SaslConfigs.SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_SECRET_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_ALGORITHM, STRING, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_ASSERTION_ALGORITHM, ConfigDef.CaseInsensitiveValidString.in("ES256", "RS256"), MEDIUM, SaslConfigs.SASL_OAUTHBEARER_ASSERTION_ALGORITHM_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_AUD, STRING, null, MEDIUM, SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_AUD_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_EXP_SECONDS, INT, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_ASSERTION_CLAIM_EXP_SECONDS, ConfigDef.Range.between(0, 86400), LOW, SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_EXP_SECONDS_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_ISS, STRING, null, MEDIUM, SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_ISS_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_JTI_INCLUDE, BOOLEAN, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_ASSERTION_CLAIM_JTI_INCLUDE, MEDIUM, SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_JTI_INCLUDE_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_NBF_SECONDS, INT, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_ASSERTION_CLAIM_NBF_SECONDS, ConfigDef.Range.between(0, 3600), LOW, SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_NBF_SECONDS_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_SUB, STRING, null, MEDIUM, SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_SUB_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_FILE, STRING, null, MEDIUM, SaslConfigs.SASL_OAUTHBEARER_ASSERTION_FILE_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_FILE, STRING, null, MEDIUM, SaslConfigs.SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_FILE_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_PASSPHRASE, PASSWORD, null, MEDIUM, SaslConfigs.SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_PASSPHRASE_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_ASSERTION_TEMPLATE_FILE, STRING, null, MEDIUM, SaslConfigs.SASL_OAUTHBEARER_ASSERTION_TEMPLATE_FILE_DOC) .define(SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME, STRING, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME, LOW, SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DOC) .define(SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME, STRING, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_SUB_CLAIM_NAME, LOW, SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME_DOC) .define(SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, STRING, null, MEDIUM, SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_DOC) diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerJwtValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/BrokerJwtValidator.java similarity index 56% rename from clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerJwtValidator.java rename to clients/src/main/java/org/apache/kafka/common/security/oauthbearer/BrokerJwtValidator.java index 74ad4765222..23ebca74050 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerJwtValidator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/BrokerJwtValidator.java @@ -15,9 +15,14 @@ * limitations under the License. */ -package org.apache.kafka.common.security.oauthbearer.internals.secured; +package org.apache.kafka.common.security.oauthbearer; -import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken; +import org.apache.kafka.common.security.oauthbearer.internals.secured.BasicOAuthBearerToken; +import org.apache.kafka.common.security.oauthbearer.internals.secured.ClaimValidationUtils; +import org.apache.kafka.common.security.oauthbearer.internals.secured.CloseableVerificationKeyResolver; +import org.apache.kafka.common.security.oauthbearer.internals.secured.ConfigurationUtils; +import org.apache.kafka.common.security.oauthbearer.internals.secured.SerializedJwt; +import org.apache.kafka.common.security.oauthbearer.internals.secured.VerificationKeyResolverFactory; import org.jose4j.jwt.JwtClaims; import org.jose4j.jwt.MalformedClaimException; @@ -27,14 +32,23 @@ import org.jose4j.jwt.consumer.InvalidJwtException; import org.jose4j.jwt.consumer.JwtConsumer; import org.jose4j.jwt.consumer.JwtConsumerBuilder; import org.jose4j.jwt.consumer.JwtContext; -import org.jose4j.keys.resolvers.VerificationKeyResolver; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Collection; import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; import java.util.Set; +import javax.security.auth.login.AppConfigurationEntry; + +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME; import static org.jose4j.jwa.AlgorithmConstraints.DISALLOW_NONE; /** @@ -43,16 +57,18 @@ import static org.jose4j.jwa.AlgorithmConstraints.DISALLOW_NONE; * from the client, but ultimately from posting the client credentials to the OAuth/OIDC provider's * token endpoint. * - * The validation steps performed (primary by the jose4j library) are: + * The validation steps performed (primarily by the jose4j library) are: * *
    *
  1. * Basic structural validation of the b64token value as defined in * RFC 6750 Section 2.1 *
  2. - *
  3. Basic conversion of the token into an in-memory data structure
  4. *
  5. - * Presence of scope, exp, subject, iss, and + * Basic conversion of the token into an in-memory data structure + *
  6. + *
  7. + * Presence of scope, exp, subject, iss, and * iat claims *
  8. *
  9. @@ -61,63 +77,46 @@ import static org.jose4j.jwa.AlgorithmConstraints.DISALLOW_NONE; *
  10. *
*/ - public class BrokerJwtValidator implements JwtValidator { private static final Logger log = LoggerFactory.getLogger(BrokerJwtValidator.class); - private final JwtConsumer jwtConsumer; + private final Optional verificationKeyResolverOpt; - private final String scopeClaimName; + private JwtConsumer jwtConsumer; - private final String subClaimName; + private String scopeClaimName; + + private String subClaimName; /** - * Creates a new {@code BrokerJwtValidator} that will be used by the broker for more - * thorough validation of the JWT. - * - * @param clockSkew The optional value (in seconds) to allow for differences - * between the time of the OAuth/OIDC identity provider and - * the broker. If null is provided, the broker - * and the OAUth/OIDC identity provider are assumed to have - * very close clock settings. - * @param expectedAudiences The (optional) set the broker will use to verify that - * the JWT was issued for one of the expected audiences. - * The JWT will be inspected for the standard OAuth - * aud claim and if this value is set, the - * broker will match the value from JWT's aud - * claim to see if there is an exact match. If there is no - * match, the broker will reject the JWT and authentication - * will fail. May be null to not perform any - * check to verify the JWT's aud claim matches any - * fixed set of known/expected audiences. - * @param expectedIssuer The (optional) value for the broker to use to verify that - * the JWT was created by the expected issuer. The JWT will - * be inspected for the standard OAuth iss claim - * and if this value is set, the broker will match it - * exactly against what is in the JWT's iss - * claim. If there is no match, the broker will reject the JWT - * and authentication will fail. May be null to not - * perform any check to verify the JWT's iss claim - * matches a specific issuer. - * @param verificationKeyResolver jose4j-based {@link VerificationKeyResolver} that is used - * to validate the signature matches the contents of the header - * and payload - * @param scopeClaimName Name of the scope claim to use; must be non-null - * @param subClaimName Name of the subject claim to use; must be - * non-null - * - * @see JwtConsumerBuilder - * @see JwtConsumer - * @see VerificationKeyResolver + * A public, no-args constructor is necessary for instantiation via configuration. */ + public BrokerJwtValidator() { + this.verificationKeyResolverOpt = Optional.empty(); + } + + /* + * Package-visible for testing. + */ + BrokerJwtValidator(CloseableVerificationKeyResolver verificationKeyResolver) { + this.verificationKeyResolverOpt = Optional.of(verificationKeyResolver); + } + + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism); + List expectedAudiencesList = cu.get(SASL_OAUTHBEARER_EXPECTED_AUDIENCE); + Set expectedAudiences = expectedAudiencesList != null ? Set.copyOf(expectedAudiencesList) : null; + Integer clockSkew = cu.validateInteger(SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS, false); + String expectedIssuer = cu.validateString(SASL_OAUTHBEARER_EXPECTED_ISSUER, false); + String scopeClaimName = cu.validateString(SASL_OAUTHBEARER_SCOPE_CLAIM_NAME); + String subClaimName = cu.validateString(SASL_OAUTHBEARER_SUB_CLAIM_NAME); + + CloseableVerificationKeyResolver verificationKeyResolver = verificationKeyResolverOpt.orElseGet( + () -> VerificationKeyResolverFactory.get(configs, saslMechanism, jaasConfigEntries) + ); - public BrokerJwtValidator(Integer clockSkew, - Set expectedAudiences, - String expectedIssuer, - VerificationKeyResolver verificationKeyResolver, - String scopeClaimName, - String subClaimName) { final JwtConsumerBuilder jwtConsumerBuilder = new JwtConsumerBuilder(); if (clockSkew != null) @@ -145,11 +144,11 @@ public class BrokerJwtValidator implements JwtValidator { * * @param accessToken Non-null JWT access token * @return {@link OAuthBearerToken} - * @throws ValidateException Thrown on errors performing validation of given token + * @throws JwtValidatorException Thrown on errors performing validation of given token */ @SuppressWarnings("unchecked") - public OAuthBearerToken validate(String accessToken) throws ValidateException { + public OAuthBearerToken validate(String accessToken) throws JwtValidatorException { SerializedJwt serializedJwt = new SerializedJwt(accessToken); JwtContext jwt; @@ -157,7 +156,7 @@ public class BrokerJwtValidator implements JwtValidator { try { jwt = jwtConsumer.process(serializedJwt.getToken()); } catch (InvalidJwtException e) { - throw new ValidateException(String.format("Could not validate the access token: %s", e.getMessage()), e); + throw new JwtValidatorException(String.format("Could not validate the access token: %s", e.getMessage()), e); } JwtClaims claims = jwt.getJwtClaims(); @@ -190,13 +189,13 @@ public class BrokerJwtValidator implements JwtValidator { issuedAt); } - private T getClaim(ClaimSupplier supplier, String claimName) throws ValidateException { + private T getClaim(ClaimSupplier supplier, String claimName) throws JwtValidatorException { try { T value = supplier.get(); log.debug("getClaim - {}: {}", claimName, value); return value; } catch (MalformedClaimException e) { - throw new ValidateException(String.format("Could not extract the '%s' claim from the access token", claimName), e); + throw new JwtValidatorException(String.format("Could not extract the '%s' claim from the access token", claimName), e); } } @@ -205,5 +204,4 @@ public class BrokerJwtValidator implements JwtValidator { T get() throws MalformedClaimException; } - } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/ClientCredentialsJwtRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/ClientCredentialsJwtRetriever.java new file mode 100644 index 00000000000..627434f6d3c --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/ClientCredentialsJwtRetriever.java @@ -0,0 +1,249 @@ +/* + * 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.kafka.common.security.oauthbearer; + +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.security.oauthbearer.internals.secured.ClientCredentialsRequestFormatter; +import org.apache.kafka.common.security.oauthbearer.internals.secured.ConfigurationUtils; +import org.apache.kafka.common.security.oauthbearer.internals.secured.HttpJwtRetriever; +import org.apache.kafka.common.security.oauthbearer.internals.secured.HttpRequestFormatter; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JaasOptionsUtils; +import org.apache.kafka.common.utils.Utils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.function.Function; + +import javax.security.auth.login.AppConfigurationEntry; + +import static org.apache.kafka.common.config.SaslConfigs.DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE; +import static org.apache.kafka.common.config.SaslConfigs.SASL_JAAS_CONFIG; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_HEADER_URLENCODE; +import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_ID_CONFIG; +import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_SECRET_CONFIG; +import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.SCOPE_CONFIG; + +/** + * {@code ClientCredentialsJwtRetriever} is a {@link JwtRetriever} that performs the steps to request + * a JWT from an OAuth/OIDC identity provider using the client_credentials grant type. This + * grant type is commonly used for non-interactive "service accounts" where there is no user available + * to interactively supply credentials. + * + *

+ * + * This {@code JwtRetriever} is enabled by specifying its class name in the Kafka configuration. + * For client use, specify the class name in the sasl.oauthbearer.jwt.retriever.class + * configuration like so: + * + *

+ * sasl.oauthbearer.jwt.retriever.class=org.apache.kafka.common.security.oauthbearer.ClientCredentialsJwtRetriever
+ * 
+ * + *

+ * + * If using this {@code JwtRetriever} on the broker side (for inter-broker communication), the configuration + * should be specified with a listener-based property: + * + *

+ * listener.name.<listener name>.oauthbearer.sasl.oauthbearer.jwt.retriever.class=org.apache.kafka.common.security.oauthbearer.ClientCredentialsJwtRetriever
+ * 
+ * + *

+ * + * The {@code ClientCredentialsJwtRetriever} also uses the following configuration: + * + *

    + *
  • sasl.oauthbearer.client.credentials.client.id
  • + *
  • sasl.oauthbearer.client.credentials.client.secret
  • + *
  • sasl.oauthbearer.scope
  • + *
  • sasl.oauthbearer.token.endpoint.url
  • + *
+ * + * Please refer to the official Apache Kafka documentation for more information on these, and related configuration. + * + *

+ * + * Previous versions of this implementation used sasl.jaas.config to specify attributes such + * as clientId, clientSecret, and scope. These will still work, but + * if the configuration for each of these is specified, it will be used instead of the JAAS option. + * + *

+ * + * Here's an example of the JAAS configuration for a Kafka client: + * + *

+ * sasl.jaas.config=org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule required ;
+ *
+ * sasl.oauthbearer.client.credentials.client.id=jdoe
+ * sasl.oauthbearer.client.credentials.client.secret=$3cr3+
+ * sasl.oauthbearer.jwt.retriever.class=org.apache.kafka.common.security.oauthbearer.ClientCredentialsJwtRetriever
+ * sasl.oauthbearer.scope=my-application-scope
+ * sasl.oauthbearer.token.endpoint.url=https://example.com/oauth2/v1/token
+ * 
+ */ +public class ClientCredentialsJwtRetriever implements JwtRetriever { + + private static final Logger LOG = LoggerFactory.getLogger(ClientCredentialsJwtRetriever.class); + + private HttpJwtRetriever delegate; + + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism); + JaasOptionsUtils jou = new JaasOptionsUtils(saslMechanism, jaasConfigEntries); + + ConfigOrJaas configOrJaas = new ConfigOrJaas(cu, jou); + String clientId = configOrJaas.clientId(); + String clientSecret = configOrJaas.clientSecret(); + String scope = configOrJaas.scope(); + boolean urlencodeHeader = validateUrlencodeHeader(cu); + + HttpRequestFormatter requestFormatter = new ClientCredentialsRequestFormatter( + clientId, + clientSecret, + scope, + urlencodeHeader + ); + + delegate = new HttpJwtRetriever(requestFormatter); + delegate.configure(configs, saslMechanism, jaasConfigEntries); + } + + @Override + public String retrieve() throws JwtRetrieverException { + if (delegate == null) + throw new IllegalStateException("JWT retriever delegate is null; please call configure() first"); + + return delegate.retrieve(); + } + + @Override + public void close() throws IOException { + Utils.closeQuietly(delegate, "JWT retriever delegate"); + } + + /** + * In some cases, the incoming {@link Map} doesn't contain a value for + * {@link SaslConfigs#SASL_OAUTHBEARER_HEADER_URLENCODE}. Returning {@code null} from {@link Map#get(Object)} + * will cause a {@link NullPointerException} when it is later unboxed. + * + *

+ * + * This utility method ensures that we have a non-{@code null} value to use in the + * {@link HttpJwtRetriever} constructor. + */ + static boolean validateUrlencodeHeader(ConfigurationUtils configurationUtils) { + Boolean urlencodeHeader = configurationUtils.get(SASL_OAUTHBEARER_HEADER_URLENCODE); + return Objects.requireNonNullElse(urlencodeHeader, DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE); + } + + /** + * Retrieves the values first from configuration, then falls back to JAAS, and, if required, throws an error. + */ + private static class ConfigOrJaas { + + private final ConfigurationUtils cu; + private final JaasOptionsUtils jou; + + private ConfigOrJaas(ConfigurationUtils cu, JaasOptionsUtils jou) { + this.cu = cu; + this.jou = jou; + } + + private String clientId() { + return getValue( + CLIENT_ID_CONFIG, + "clientId", + true, + cu::validateString, + jou::validateString + ); + } + + private String clientSecret() { + return getValue( + CLIENT_SECRET_CONFIG, + "clientSecret", + true, + cu::validatePassword, + jou::validateString + ); + } + + private String scope() { + return getValue( + SCOPE_CONFIG, + "scope", + false, + cu::validateString, + jou::validateString + ); + } + + private String getValue(String configName, + String jaasName, + boolean isRequired, + Function configValueGetter, + Function jaasValueGetter) { + boolean isPresentInConfig = cu.containsKey(configName); + boolean isPresentInJaas = jou.containsKey(jaasName); + + if (isPresentInConfig) { + if (isPresentInJaas) { + // Log if the user is using the deprecated JAAS option. + LOG.warn( + "Both the OAuth configuration {} as well as the JAAS option {} (from the {} configuration) were provided. " + + "Since the {} JAAS option is deprecated, it will be ignored and the value from the {} configuration will be used. " + + "Please update your configuration to only use {}.", + configName, + jaasName, + SASL_JAAS_CONFIG, + jaasName, + configName, + configName + ); + } + + return configValueGetter.apply(configName); + } else if (isPresentInJaas) { + String value = jaasValueGetter.apply(jaasName); + + // Log if the user is using the deprecated JAAS option. + LOG.warn( + "The OAuth JAAS option {} was configured in {}, but that JAAS option is deprecated and will be removed. " + + "Please update your configuration to use the {} configuration instead.", + jaasName, + SASL_JAAS_CONFIG, + configName + ); + + return value; + } else if (isRequired) { + throw new ConfigException(configName, null); + } else { + return null; + } + } + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientJwtValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/ClientJwtValidator.java similarity index 76% rename from clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientJwtValidator.java rename to clients/src/main/java/org/apache/kafka/common/security/oauthbearer/ClientJwtValidator.java index 1dee4671d39..53cd88f24dd 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientJwtValidator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/ClientJwtValidator.java @@ -15,9 +15,12 @@ * limitations under the License. */ -package org.apache.kafka.common.security.oauthbearer.internals.secured; +package org.apache.kafka.common.security.oauthbearer; -import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken; +import org.apache.kafka.common.security.oauthbearer.internals.secured.BasicOAuthBearerToken; +import org.apache.kafka.common.security.oauthbearer.internals.secured.ClaimValidationUtils; +import org.apache.kafka.common.security.oauthbearer.internals.secured.ConfigurationUtils; +import org.apache.kafka.common.security.oauthbearer.internals.secured.SerializedJwt; import org.apache.kafka.common.security.oauthbearer.internals.unsecured.OAuthBearerIllegalTokenException; import org.apache.kafka.common.security.oauthbearer.internals.unsecured.OAuthBearerUnsecuredJws; @@ -26,11 +29,16 @@ import org.slf4j.LoggerFactory; import java.util.Collection; import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.Set; +import javax.security.auth.login.AppConfigurationEntry; + import static org.apache.kafka.common.config.SaslConfigs.DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME; import static org.apache.kafka.common.config.SaslConfigs.DEFAULT_SASL_OAUTHBEARER_SUB_CLAIM_NAME; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME; /** * {@code ClientJwtValidator} is an implementation of {@link JwtValidator} that is used @@ -58,21 +66,21 @@ public class ClientJwtValidator implements JwtValidator { public static final String ISSUED_AT_CLAIM_NAME = "iat"; - private final String scopeClaimName; + private String scopeClaimName; - private final String subClaimName; + private String subClaimName; - /** - * Creates a new {@code ClientJwtValidator} that will be used by the client for lightweight - * validation of the JWT. - * - * @param scopeClaimName Name of the scope claim to use; must be non-null - * @param subClaimName Name of the subject claim to use; must be non-null - */ - - public ClientJwtValidator(String scopeClaimName, String subClaimName) { - this.scopeClaimName = ClaimValidationUtils.validateClaimNameOverride(DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME, scopeClaimName); - this.subClaimName = ClaimValidationUtils.validateClaimNameOverride(DEFAULT_SASL_OAUTHBEARER_SUB_CLAIM_NAME, subClaimName); + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism); + this.scopeClaimName = ClaimValidationUtils.validateClaimNameOverride( + DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME, + cu.get(SASL_OAUTHBEARER_SCOPE_CLAIM_NAME) + ); + this.subClaimName = ClaimValidationUtils.validateClaimNameOverride( + DEFAULT_SASL_OAUTHBEARER_SUB_CLAIM_NAME, + cu.get(SASL_OAUTHBEARER_SUB_CLAIM_NAME) + ); } /** @@ -81,18 +89,18 @@ public class ClientJwtValidator implements JwtValidator { * * @param accessToken Non-null JWT access token * @return {@link OAuthBearerToken} - * @throws ValidateException Thrown on errors performing validation of given token + * @throws JwtValidatorException Thrown on errors performing validation of given token */ @SuppressWarnings("unchecked") - public OAuthBearerToken validate(String accessToken) throws ValidateException { + public OAuthBearerToken validate(String accessToken) throws JwtValidatorException { SerializedJwt serializedJwt = new SerializedJwt(accessToken); Map payload; try { payload = OAuthBearerUnsecuredJws.toMap(serializedJwt.getPayload()); } catch (OAuthBearerIllegalTokenException e) { - throw new ValidateException(String.format("Could not validate the access token: %s", e.getMessage()), e); + throw new JwtValidatorException(String.format("Could not validate the access token: %s", e.getMessage()), e); } Object scopeRaw = getClaim(payload, scopeClaimName); diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtRetriever.java new file mode 100644 index 00000000000..5044a36aba5 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtRetriever.java @@ -0,0 +1,90 @@ +/* + * 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.kafka.common.security.oauthbearer; + +import org.apache.kafka.common.security.oauthbearer.internals.secured.ClientCredentialsRequestFormatter; +import org.apache.kafka.common.security.oauthbearer.internals.secured.ConfigurationUtils; +import org.apache.kafka.common.utils.Utils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.URL; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +import javax.security.auth.login.AppConfigurationEntry; + +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL; + +/** + * {@code DefaultJwtRetriever} instantiates and delegates {@link JwtRetriever} API calls to an embedded implementation + * based on configuration: + * + *

    + *
  • + * If the value of sasl.oauthbearer.token.endpoint.url is set to a value that starts with the + * file protocol (e.g. file:/tmp/path/to/a/static-jwt.json), an instance of + * {@link FileJwtRetriever} will be used as the underlying {@link JwtRetriever}. Otherwise, the URL is + * assumed to be an HTTP/HTTPS-based URL, and an instance of {@link ClientCredentialsRequestFormatter} will + * be created and used. + *
  • + *
+ * + * The configuration required by the individual {@code JwtRetriever} classes will likely differ. Please refer to the + * official Apache Kafka documentation for more information on these, and related configuration. + */ +public class DefaultJwtRetriever implements JwtRetriever { + + private static final Logger LOG = LoggerFactory.getLogger(DefaultJwtRetriever.class); + + private JwtRetriever delegate; + + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism); + URL tokenEndpointUrl = cu.validateUrl(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL); + + if (tokenEndpointUrl.getProtocol().toLowerCase(Locale.ROOT).equals("file")) + delegate = new FileJwtRetriever(); + else + delegate = new ClientCredentialsJwtRetriever(); + + LOG.debug("Created instance of {} as delegate", delegate.getClass().getName()); + delegate.configure(configs, saslMechanism, jaasConfigEntries); + } + + @Override + public String retrieve() throws JwtRetrieverException { + if (delegate == null) + throw new IllegalStateException("JWT retriever delegate is null; please call configure() first"); + + return delegate.retrieve(); + } + + @Override + public void close() throws IOException { + Utils.closeQuietly(delegate, "JWT retriever delegate"); + } + + JwtRetriever delegate() { + return delegate; + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtValidator.java new file mode 100644 index 00000000000..478a0fdc916 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtValidator.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.kafka.common.security.oauthbearer; + +import org.apache.kafka.common.security.oauthbearer.internals.secured.CloseableVerificationKeyResolver; +import org.apache.kafka.common.utils.Utils; + +import org.jose4j.keys.resolvers.VerificationKeyResolver; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import javax.security.auth.login.AppConfigurationEntry; + +/** + * This {@link JwtValidator} uses the delegation approach, instantiating and delegating calls to a + * more concrete implementation. The underlying implementation is determined by the presence/absence + * of the {@link VerificationKeyResolver}: if it's present, a {@link BrokerJwtValidator} is + * created, otherwise a {@link ClientJwtValidator} is created. + */ +public class DefaultJwtValidator implements JwtValidator { + + private final Optional verificationKeyResolver; + + private JwtValidator delegate; + + public DefaultJwtValidator() { + this.verificationKeyResolver = Optional.empty(); + } + + public DefaultJwtValidator(CloseableVerificationKeyResolver verificationKeyResolver) { + this.verificationKeyResolver = Optional.of(verificationKeyResolver); + } + + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + if (verificationKeyResolver.isPresent()) { + delegate = new BrokerJwtValidator(verificationKeyResolver.get()); + } else { + delegate = new ClientJwtValidator(); + } + + delegate.configure(configs, saslMechanism, jaasConfigEntries); + } + + @Override + public OAuthBearerToken validate(String accessToken) throws JwtValidatorException { + if (delegate == null) + throw new IllegalStateException("JWT validator delegate is null; please call configure() first"); + + return delegate.validate(accessToken); + } + + @Override + public void close() throws IOException { + Utils.closeQuietly(delegate, "JWT validator delegate"); + } + + JwtValidator delegate() { + return delegate; + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/FileJwtRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/FileJwtRetriever.java new file mode 100644 index 00000000000..eeaee1cfb53 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/FileJwtRetriever.java @@ -0,0 +1,59 @@ +/* + * 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.kafka.common.security.oauthbearer; + +import org.apache.kafka.common.security.oauthbearer.internals.secured.CachedFile; +import org.apache.kafka.common.security.oauthbearer.internals.secured.ConfigurationUtils; + +import java.io.File; +import java.util.List; +import java.util.Map; + +import javax.security.auth.login.AppConfigurationEntry; + +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.CachedFile.RefreshPolicy.lastModifiedPolicy; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.CachedFile.STRING_JSON_VALIDATING_TRANSFORMER; + +/** + * FileJwtRetriever is an {@link JwtRetriever} that will load the contents + * of a file, interpreting them as a JWT access key in the serialized form. + */ +public class FileJwtRetriever implements JwtRetriever { + + private CachedFile jwtFile; + + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism); + File file = cu.validateFileUrl(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL); + jwtFile = new CachedFile<>(file, STRING_JSON_VALIDATING_TRANSFORMER, lastModifiedPolicy()); + } + + @Override + public String retrieve() throws JwtRetrieverException { + if (jwtFile == null) + throw new IllegalStateException("JWT is null; please call configure() first"); + + try { + return jwtFile.transformed(); + } catch (Exception e) { + throw new JwtRetrieverException(e); + } + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtBearerJwtRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtBearerJwtRetriever.java new file mode 100644 index 00000000000..b6eb3a39372 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtBearerJwtRetriever.java @@ -0,0 +1,181 @@ +/* + * 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.kafka.common.security.oauthbearer; + +import org.apache.kafka.common.security.oauthbearer.internals.secured.ConfigurationUtils; +import org.apache.kafka.common.security.oauthbearer.internals.secured.HttpJwtRetriever; +import org.apache.kafka.common.security.oauthbearer.internals.secured.HttpRequestFormatter; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtBearerRequestFormatter; +import org.apache.kafka.common.security.oauthbearer.internals.secured.assertion.AssertionCreator; +import org.apache.kafka.common.security.oauthbearer.internals.secured.assertion.AssertionJwtTemplate; +import org.apache.kafka.common.security.oauthbearer.internals.secured.assertion.DefaultAssertionCreator; +import org.apache.kafka.common.security.oauthbearer.internals.secured.assertion.FileAssertionCreator; +import org.apache.kafka.common.security.oauthbearer.internals.secured.assertion.StaticAssertionJwtTemplate; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.Supplier; + +import javax.security.auth.login.AppConfigurationEntry; + +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_ASSERTION_ALGORITHM; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_ASSERTION_FILE; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_FILE; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_PASSPHRASE; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SCOPE; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.assertion.AssertionUtils.layeredAssertionJwtTemplate; + +/** + * {@code JwtBearerJwtRetriever} is a {@link JwtRetriever} that performs the steps to request + * a JWT from an OAuth/OIDC identity provider using the urn:ietf:params:oauth:grant-type:jwt-bearer + * grant type. This grant type is used for machine-to-machine "service accounts". + * + *

+ * + * This {@code JwtRetriever} is enabled by specifying its class name in the Kafka configuration. + * For client use, specify the class name in the sasl.oauthbearer.jwt.retriever.class + * configuration like so: + * + *

+ * sasl.oauthbearer.jwt.retriever.class=org.apache.kafka.common.security.oauthbearer.JwtBearerJwtRetriever
+ * 
+ * + *

+ * + * If using this {@code JwtRetriever} on the broker side (for inter-broker communication), the configuration + * should be specified with a listener-based property: + * + *

+ * listener.name.<listener name>.oauthbearer.sasl.oauthbearer.jwt.retriever.class=org.apache.kafka.common.security.oauthbearer.JwtBearerJwtRetriever
+ * 
+ * + *

+ * + * The {@code JwtBearerJwtRetriever} also uses the following configuration: + * + *

    + *
  • sasl.oauthbearer.assertion.algorithm
  • + *
  • sasl.oauthbearer.assertion.claim.aud
  • + *
  • sasl.oauthbearer.assertion.claim.exp.seconds
  • + *
  • sasl.oauthbearer.assertion.claim.iss
  • + *
  • sasl.oauthbearer.assertion.claim.jti.include
  • + *
  • sasl.oauthbearer.assertion.claim.nbf.seconds
  • + *
  • sasl.oauthbearer.assertion.claim.sub
  • + *
  • sasl.oauthbearer.assertion.file
  • + *
  • sasl.oauthbearer.assertion.private.key.file
  • + *
  • sasl.oauthbearer.assertion.private.key.passphrase
  • + *
  • sasl.oauthbearer.assertion.template.file
  • + *
  • sasl.oauthbearer.jwt.retriever.class
  • + *
  • sasl.oauthbearer.scope
  • + *
  • sasl.oauthbearer.token.endpoint.url
  • + *
+ * + * Please refer to the official Apache Kafka documentation for more information on these, and related, configuration. + * + *

+ * + * Here's an example of the JAAS configuration for a Kafka client: + * + *

+ * sasl.jaas.config=org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule required ;
+ *
+ * sasl.oauthbearer.assertion.algorithm=RS256
+ * sasl.oauthbearer.assertion.claim.aud=my-application-audience
+ * sasl.oauthbearer.assertion.claim.exp.seconds=600
+ * sasl.oauthbearer.assertion.claim.iss=my-oauth-issuer
+ * sasl.oauthbearer.assertion.claim.jti.include=true
+ * sasl.oauthbearer.assertion.claim.nbf.seconds=120
+ * sasl.oauthbearer.assertion.claim.sub=kafka-app-1234
+ * sasl.oauthbearer.assertion.private.key.file=/path/to/private.key
+ * sasl.oauthbearer.assertion.private.key.passphrase=$3cr3+
+ * sasl.oauthbearer.assertion.template.file=/path/to/assertion-template.json
+ * sasl.oauthbearer.jwt.retriever.class=org.apache.kafka.common.security.oauthbearer.JwtBearerJwtRetriever
+ * sasl.oauthbearer.scope=my-application-scope
+ * sasl.oauthbearer.token.endpoint.url=https://example.com/oauth2/v1/token
+ * 
+ */ +public class JwtBearerJwtRetriever implements JwtRetriever { + + private final Time time; + private HttpJwtRetriever delegate; + private AssertionJwtTemplate assertionJwtTemplate; + private AssertionCreator assertionCreator; + + public JwtBearerJwtRetriever() { + this(Time.SYSTEM); + } + + public JwtBearerJwtRetriever(Time time) { + this.time = time; + } + + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism); + + String scope = cu.validateString(SASL_OAUTHBEARER_SCOPE, false); + + if (cu.validateString(SASL_OAUTHBEARER_ASSERTION_FILE, false) != null) { + File assertionFile = cu.validateFile(SASL_OAUTHBEARER_ASSERTION_FILE); + assertionCreator = new FileAssertionCreator(assertionFile); + assertionJwtTemplate = new StaticAssertionJwtTemplate(); + } else { + String algorithm = cu.validateString(SASL_OAUTHBEARER_ASSERTION_ALGORITHM); + File privateKeyFile = cu.validateFile(SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_FILE); + Optional passphrase = cu.containsKey(SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_PASSPHRASE) ? + Optional.of(cu.validatePassword(SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_PASSPHRASE)) : + Optional.empty(); + + assertionCreator = new DefaultAssertionCreator(algorithm, privateKeyFile, passphrase); + assertionJwtTemplate = layeredAssertionJwtTemplate(cu, time); + } + + Supplier assertionSupplier = () -> { + try { + return assertionCreator.create(assertionJwtTemplate); + } catch (Exception e) { + throw new JwtRetrieverException(e); + } + }; + + HttpRequestFormatter requestFormatter = new JwtBearerRequestFormatter(scope, assertionSupplier); + + delegate = new HttpJwtRetriever(requestFormatter); + delegate.configure(configs, saslMechanism, jaasConfigEntries); + } + + @Override + public String retrieve() throws JwtRetrieverException { + if (delegate == null) + throw new IllegalStateException("JWT retriever delegate is null; please call configure() first"); + + return delegate.retrieve(); + } + + @Override + public void close() throws IOException { + Utils.closeQuietly(assertionCreator, "JWT assertion creator"); + Utils.closeQuietly(assertionJwtTemplate, "JWT assertion template"); + Utils.closeQuietly(delegate, "JWT retriever delegate"); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtRetriever.java similarity index 73% rename from clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtRetriever.java rename to clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtRetriever.java index b8991250df0..7510a27883c 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtRetriever.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtRetriever.java @@ -15,10 +15,11 @@ * limitations under the License. */ -package org.apache.kafka.common.security.oauthbearer.internals.secured; +package org.apache.kafka.common.security.oauthbearer; + +import org.apache.kafka.common.security.oauthbearer.internals.secured.HttpJwtRetriever; +import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerConfigurable; -import java.io.Closeable; -import java.io.IOException; /** * A JwtRetriever is the internal API by which the login module will @@ -34,7 +35,7 @@ import java.io.IOException; * @see FileJwtRetriever */ -public interface JwtRetriever extends Initable, Closeable { +public interface JwtRetriever extends OAuthBearerConfigurable { /** * Retrieves a JWT access token in its serialized three-part form. The implementation @@ -48,21 +49,8 @@ public interface JwtRetriever extends Initable, Closeable { * * @return Non-null JWT access token string * - * @throws IOException Thrown on errors related to IO during retrieval + * @throws JwtRetrieverException Thrown on errors related to IO during retrieval */ - String retrieve() throws IOException; - - /** - * Lifecycle method to perform a clean shutdown of the retriever. This must - * be performed by the caller to ensure the correct state, freeing up and releasing any - * resources performed in {@link #init()}. - * - * @throws IOException Thrown on errors related to IO during closure - */ - - default void close() throws IOException { - // This method left intentionally blank. - } - + String retrieve() throws JwtRetrieverException; } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtRetrieverException.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtRetrieverException.java new file mode 100644 index 00000000000..a83844fb415 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtRetrieverException.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.kafka.common.security.oauthbearer; + +import org.apache.kafka.common.KafkaException; + +/** + * A {@code JwtRetrieverException} is thrown in cases where the JWT cannot be retrieved. + * + * @see JwtRetriever#retrieve() + */ +public class JwtRetrieverException extends KafkaException { + + public JwtRetrieverException(String message) { + super(message); + } + + public JwtRetrieverException(Throwable cause) { + super(cause); + } + + public JwtRetrieverException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtValidator.java similarity index 81% rename from clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtValidator.java rename to clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtValidator.java index 82ba10652a1..2d74e414913 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtValidator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtValidator.java @@ -15,12 +15,10 @@ * limitations under the License. */ -package org.apache.kafka.common.security.oauthbearer.internals.secured; +package org.apache.kafka.common.security.oauthbearer; -import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken; +import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerConfigurable; -import java.io.Closeable; -import java.io.IOException; /** * An instance of JwtValidator acts as a function object that, given an access @@ -48,7 +46,7 @@ import java.io.IOException; * contents and verify the signature */ -public interface JwtValidator extends Initable, Closeable { +public interface JwtValidator extends OAuthBearerConfigurable { /** * Accepts an OAuth JWT access token in base-64 encoded format, validates, and returns an @@ -58,15 +56,8 @@ public interface JwtValidator extends Initable, Closeable { * * @return {@link OAuthBearerToken} * - * @throws ValidateException Thrown on errors performing validation of given token + * @throws JwtValidatorException Thrown on errors performing validation of given token */ - OAuthBearerToken validate(String accessToken) throws ValidateException; - - /** - * Closes any resources that were initialized by {@link #init()}. - */ - default void close() throws IOException { - // Do nothing... - } + OAuthBearerToken validate(String accessToken) throws JwtValidatorException; } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ValidateException.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtValidatorException.java similarity index 62% rename from clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ValidateException.java rename to clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtValidatorException.java index 8c107abc831..6aef68aaf4b 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ValidateException.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtValidatorException.java @@ -15,33 +15,32 @@ * limitations under the License. */ -package org.apache.kafka.common.security.oauthbearer.internals.secured; +package org.apache.kafka.common.security.oauthbearer; import org.apache.kafka.common.KafkaException; import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; /** - * ValidateException is thrown in cases where a JWT access token cannot be determined to be - * valid for one reason or another. It is intended to be used when errors arise within the - * processing of a {@link javax.security.auth.callback.CallbackHandler#handle(Callback[])}. - * This error, however, is not thrown from that method directly. + * A {@code JwtValidatorException} is thrown in cases where the validity of a JWT cannot be + * determined. It is intended to be used when errors arise within the processing of a + * {@link CallbackHandler#handle(Callback[])}. This error, however, is not thrown from that + * method directly. * * @see JwtValidator#validate(String) */ +public class JwtValidatorException extends KafkaException { -public class ValidateException extends KafkaException { - - public ValidateException(String message) { + public JwtValidatorException(String message) { super(message); } - public ValidateException(Throwable cause) { + public JwtValidatorException(Throwable cause) { super(cause); } - public ValidateException(String message, Throwable cause) { + public JwtValidatorException(String message, Throwable cause) { super(message, cause); } - } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginCallbackHandler.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginCallbackHandler.java index 0d8701ba11d..6afd31df273 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginCallbackHandler.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginCallbackHandler.java @@ -17,19 +17,14 @@ package org.apache.kafka.common.security.oauthbearer; -import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.config.SaslConfigs; import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler; import org.apache.kafka.common.security.auth.SaslExtensions; import org.apache.kafka.common.security.auth.SaslExtensionsCallback; import org.apache.kafka.common.security.oauthbearer.internals.OAuthBearerClientInitialResponse; -import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtRetriever; -import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtValidator; import org.apache.kafka.common.security.oauthbearer.internals.secured.JaasOptionsUtils; -import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtRetriever; -import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtValidator; -import org.apache.kafka.common.security.oauthbearer.internals.secured.ValidateException; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; @@ -46,13 +41,14 @@ import javax.security.auth.login.AppConfigurationEntry; import javax.security.sasl.SaslException; import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.ConfigurationUtils.getConfiguredInstance; /** *

* OAuthBearerLoginCallbackHandler is an {@link AuthenticateCallbackHandler} that * accepts {@link OAuthBearerTokenCallback} and {@link SaslExtensionsCallback} callbacks to * perform the steps to request a JWT from an OAuth/OIDC provider using the - * clientcredentials. This grant type is commonly used for non-interactive + * client_credentials. This grant type is commonly used for non-interactive * "service accounts" where there is no user available to interactively supply credentials. *

* @@ -186,31 +182,39 @@ public class OAuthBearerLoginCallbackHandler implements AuthenticateCallbackHand @Override public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { - Map moduleOptions = JaasOptionsUtils.getOptions(saslMechanism, jaasConfigEntries); - JwtRetriever jwtRetriever = new DefaultJwtRetriever(configs, saslMechanism, moduleOptions); - JwtValidator jwtValidator = new DefaultJwtValidator(configs, saslMechanism); - init(moduleOptions, jwtRetriever, jwtValidator); + moduleOptions = JaasOptionsUtils.getOptions(saslMechanism, jaasConfigEntries); + jwtRetriever = getConfiguredInstance( + configs, + saslMechanism, + jaasConfigEntries, + SaslConfigs.SASL_OAUTHBEARER_JWT_RETRIEVER_CLASS, + JwtRetriever.class + ); + + jwtValidator = getConfiguredInstance( + configs, + saslMechanism, + jaasConfigEntries, + SaslConfigs.SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS, + JwtValidator.class + ); } /* * Package-visible for testing. */ - void init(Map moduleOptions, JwtRetriever jwtRetriever, JwtValidator jwtValidator) { - this.moduleOptions = moduleOptions; + void configure(Map configs, + String saslMechanism, + List jaasConfigEntries, + JwtRetriever jwtRetriever, + JwtValidator jwtValidator) { + this.moduleOptions = JaasOptionsUtils.getOptions(saslMechanism, jaasConfigEntries); + this.jwtRetriever = jwtRetriever; + this.jwtRetriever.configure(configs, saslMechanism, jaasConfigEntries); + this.jwtValidator = jwtValidator; - - try { - this.jwtRetriever.init(); - } catch (IOException e) { - throw new KafkaException("The OAuth login callback encountered an error when initializing the JwtRetriever", e); - } - - try { - this.jwtValidator.init(); - } catch (IOException e) { - throw new KafkaException("The OAuth login callback encountered an error when initializing the JwtValidator", e); - } + this.jwtValidator.configure(configs, saslMechanism, jaasConfigEntries); } @Override @@ -241,7 +245,7 @@ public class OAuthBearerLoginCallbackHandler implements AuthenticateCallbackHand try { OAuthBearerToken token = jwtValidator.validate(accessToken); callback.token(token); - } catch (ValidateException e) { + } catch (JwtValidatorException e) { log.warn(e.getMessage(), e); callback.error("invalid_token", e.getMessage(), null); } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandler.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandler.java index c10b7db4e24..6563d36b8b6 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandler.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandler.java @@ -17,35 +17,24 @@ package org.apache.kafka.common.security.oauthbearer; -import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.config.SaslConfigs; import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler; import org.apache.kafka.common.security.oauthbearer.internals.secured.CloseableVerificationKeyResolver; -import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtValidator; -import org.apache.kafka.common.security.oauthbearer.internals.secured.JaasOptionsUtils; -import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtValidator; -import org.apache.kafka.common.security.oauthbearer.internals.secured.RefreshingHttpsJwksVerificationKeyResolver; -import org.apache.kafka.common.security.oauthbearer.internals.secured.ValidateException; -import org.apache.kafka.common.security.oauthbearer.internals.secured.VerificationKeyResolverFactory; import org.apache.kafka.common.utils.Utils; -import org.jose4j.jws.JsonWebSignature; -import org.jose4j.jwx.JsonWebStructure; -import org.jose4j.lang.UnresolvableKeyException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.security.Key; -import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Objects; -import java.util.concurrent.atomic.AtomicInteger; import javax.security.auth.callback.Callback; import javax.security.auth.callback.UnsupportedCallbackException; import javax.security.auth.login.AppConfigurationEntry; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.ConfigurationUtils.getConfiguredInstance; + /** *

* OAuthBearerValidatorCallbackHandler is an {@link AuthenticateCallbackHandler} that @@ -109,53 +98,34 @@ public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallback private static final Logger log = LoggerFactory.getLogger(OAuthBearerValidatorCallbackHandler.class); - /** - * Because a {@link CloseableVerificationKeyResolver} instance can spawn threads and issue - * HTTP(S) calls ({@link RefreshingHttpsJwksVerificationKeyResolver}), we only want to create - * a new instance for each particular set of configuration. Because each set of configuration - * may have multiple instances, we want to reuse the single instance. - */ - - private static final Map VERIFICATION_KEY_RESOLVER_CACHE = new HashMap<>(); - private CloseableVerificationKeyResolver verificationKeyResolver; private JwtValidator jwtValidator; @Override public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { - Map moduleOptions = JaasOptionsUtils.getOptions(saslMechanism, jaasConfigEntries); - CloseableVerificationKeyResolver verificationKeyResolver; - - // Here's the logic which keeps our VerificationKeyResolvers down to a single instance. - synchronized (VERIFICATION_KEY_RESOLVER_CACHE) { - VerificationKeyResolverKey key = new VerificationKeyResolverKey(configs, moduleOptions); - verificationKeyResolver = VERIFICATION_KEY_RESOLVER_CACHE.computeIfAbsent(key, k -> - new RefCountingVerificationKeyResolver(VerificationKeyResolverFactory.create(configs, saslMechanism, moduleOptions))); - } - - JwtValidator jwtValidator = new DefaultJwtValidator(configs, saslMechanism, verificationKeyResolver); - init(verificationKeyResolver, jwtValidator); + jwtValidator = getConfiguredInstance( + configs, + saslMechanism, + jaasConfigEntries, + SaslConfigs.SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS, + JwtValidator.class + ); } /* * Package-visible for testing. */ - void init(CloseableVerificationKeyResolver verificationKeyResolver, JwtValidator jwtValidator) { + void configure(Map configs, + String saslMechanism, + List jaasConfigEntries, + CloseableVerificationKeyResolver verificationKeyResolver, + JwtValidator jwtValidator) { this.verificationKeyResolver = verificationKeyResolver; + this.verificationKeyResolver.configure(configs, saslMechanism, jaasConfigEntries); + this.jwtValidator = jwtValidator; - - try { - verificationKeyResolver.init(); - } catch (Exception e) { - throw new KafkaException("The OAuth validator callback encountered an error when initializing the VerificationKeyResolver", e); - } - - try { - jwtValidator.init(); - } catch (IOException e) { - throw new KafkaException("The OAuth validator callback encountered an error when initializing the JwtValidator", e); - } + this.jwtValidator.configure(configs, saslMechanism, jaasConfigEntries); } @Override @@ -187,7 +157,7 @@ public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallback try { token = jwtValidator.validate(callback.tokenValue()); callback.token(token); - } catch (ValidateException e) { + } catch (JwtValidatorException e) { log.warn(e.getMessage(), e); callback.error("invalid_token", null, null); } @@ -203,79 +173,4 @@ public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallback if (verificationKeyResolver == null || jwtValidator == null) throw new IllegalStateException(String.format("To use %s, first call the configure method", getClass().getSimpleName())); } - - /** - * VkrKey is a simple structure which encapsulates the criteria for different - * sets of configuration. This will allow us to use this object as a key in a {@link Map} - * to keep a single instance per key. - */ - - private static class VerificationKeyResolverKey { - - private final Map configs; - - private final Map moduleOptions; - - public VerificationKeyResolverKey(Map configs, Map moduleOptions) { - this.configs = configs; - this.moduleOptions = moduleOptions; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (o == null || getClass() != o.getClass()) { - return false; - } - - VerificationKeyResolverKey that = (VerificationKeyResolverKey) o; - return configs.equals(that.configs) && moduleOptions.equals(that.moduleOptions); - } - - @Override - public int hashCode() { - return Objects.hash(configs, moduleOptions); - } - - } - - /** - * RefCountingVerificationKeyResolver allows us to share a single - * {@link CloseableVerificationKeyResolver} instance between multiple - * {@link AuthenticateCallbackHandler} instances and perform the lifecycle methods the - * appropriate number of times. - */ - - private static class RefCountingVerificationKeyResolver implements CloseableVerificationKeyResolver { - - private final CloseableVerificationKeyResolver delegate; - - private final AtomicInteger count = new AtomicInteger(0); - - public RefCountingVerificationKeyResolver(CloseableVerificationKeyResolver delegate) { - this.delegate = delegate; - } - - @Override - public Key resolveKey(JsonWebSignature jws, List nestingContext) throws UnresolvableKeyException { - return delegate.resolveKey(jws, nestingContext); - } - - @Override - public void init() throws IOException { - if (count.incrementAndGet() == 1) - delegate.init(); - } - - @Override - public void close() throws IOException { - if (count.decrementAndGet() == 0) - delegate.close(); - } - - } - } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CachedFile.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CachedFile.java new file mode 100644 index 00000000000..11cfb19cf49 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CachedFile.java @@ -0,0 +1,179 @@ +/* + * 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.kafka.common.security.oauthbearer.internals.secured; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.security.oauthbearer.JwtValidatorException; +import org.apache.kafka.common.security.oauthbearer.internals.unsecured.OAuthBearerIllegalTokenException; +import org.apache.kafka.common.security.oauthbearer.internals.unsecured.OAuthBearerUnsecuredJws; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; + +/** + * {@code CachedFile} goes a little beyond the basic file caching mechanism by allowing the file to be "transformed" + * into an in-memory representation of the file contents for easier use by the caller. + * + * @param Type of the "transformed" file contents + */ +public class CachedFile { + + /** + * Function object that provides as arguments the file and its contents and returns the in-memory representation + * of the file contents. + */ + public interface Transformer { + + /** + * Transforms the raw contents into a (possibly) different representation. + * + * @param file File containing the source data + * @param contents Data from file; could be zero length but not {@code null} + */ + T transform(File file, String contents); + } + + /** + * Function object that provides as arguments the file and its metadata and returns a flag to determine if the + * file should be reloaded from disk. + */ + public interface RefreshPolicy { + + /** + * Given the {@link File} and its snapshot, determine if the file should be reloaded from disk. + */ + boolean shouldRefresh(File file, Snapshot snapshot); + + /** + * This cache refresh policy only loads the file once. + */ + static RefreshPolicy staticPolicy() { + return (file, snapshot) -> snapshot == null; + } + + /** + * This policy will refresh the cached file if the snapshot's time is older than the current timestamp. + */ + static RefreshPolicy lastModifiedPolicy() { + return (file, snapshot) -> { + if (snapshot == null) + return true; + + return file.lastModified() != snapshot.lastModified(); + }; + } + } + + /** + * No-op transformer that retains the exact file contents as a string. + */ + public static final Transformer STRING_NOOP_TRANSFORMER = (file, contents) -> contents; + + /** + * This transformer really only validates that the given file contents represent a properly-formed JWT. + * If not, a {@link OAuthBearerIllegalTokenException} or {@link JwtValidatorException} is thrown. + */ + public static final Transformer STRING_JSON_VALIDATING_TRANSFORMER = (file, contents) -> { + contents = contents.trim(); + SerializedJwt serializedJwt = new SerializedJwt(contents); + OAuthBearerUnsecuredJws.toMap(serializedJwt.getHeader()); + OAuthBearerUnsecuredJws.toMap(serializedJwt.getPayload()); + return contents; + }; + + private final File file; + private final Transformer transformer; + private final RefreshPolicy cacheRefreshPolicy; + private Snapshot snapshot; + + public CachedFile(File file, Transformer transformer, RefreshPolicy cacheRefreshPolicy) { + this.file = file; + this.transformer = transformer; + this.cacheRefreshPolicy = cacheRefreshPolicy; + this.snapshot = snapshot(); + } + + public long size() { + return snapshot().size(); + } + + public long lastModified() { + return snapshot().lastModified(); + } + + public String contents() { + return snapshot().contents(); + } + + public T transformed() { + return snapshot().transformed(); + } + + private Snapshot snapshot() { + if (cacheRefreshPolicy.shouldRefresh(file, snapshot)) { + long size = file.length(); + long lastModified = file.lastModified(); + String contents; + + try { + contents = Files.readString(file.toPath()); + } catch (IOException e) { + throw new KafkaException("Error reading the file contents of OAuth resource " + file.getPath() + " for caching"); + } + + T transformed = transformer.transform(file, contents); + snapshot = new Snapshot<>(size, lastModified, contents, transformed); + } + + return snapshot; + } + + public static class Snapshot { + + private final long size; + + private final long lastModified; + + private final String contents; + + private final T transformed; + + public Snapshot(long size, long lastModified, String contents, T transformed) { + this.size = size; + this.lastModified = lastModified; + this.contents = contents; + this.transformed = transformed; + } + + public long size() { + return size; + } + + public long lastModified() { + return lastModified; + } + + public String contents() { + return contents; + } + + public T transformed() { + return transformed; + } + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClaimValidationUtils.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClaimValidationUtils.java index 5bf5ef068ed..582b4e86f70 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClaimValidationUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClaimValidationUtils.java @@ -17,6 +17,8 @@ package org.apache.kafka.common.security.oauthbearer.internals.secured; +import org.apache.kafka.common.security.oauthbearer.JwtValidatorException; + import java.util.Collection; import java.util.Collections; import java.util.HashSet; @@ -47,14 +49,14 @@ public class ClaimValidationUtils { * @return Unmodifiable {@link Set} that includes the values of the original set, but with * each value trimmed * - * @throws ValidateException Thrown if the value is null, contains duplicates, or + * @throws JwtValidatorException Thrown if the value is null, contains duplicates, or * if any of the values in the set are null, empty, * or whitespace only */ - public static Set validateScopes(String scopeClaimName, Collection scopes) throws ValidateException { + public static Set validateScopes(String scopeClaimName, Collection scopes) throws JwtValidatorException { if (scopes == null) - throw new ValidateException(String.format("%s value must be non-null", scopeClaimName)); + throw new JwtValidatorException(String.format("%s value must be non-null", scopeClaimName)); Set copy = new HashSet<>(); @@ -62,7 +64,7 @@ public class ClaimValidationUtils { scope = validateString(scopeClaimName, scope); if (copy.contains(scope)) - throw new ValidateException(String.format("%s value must not contain duplicates - %s already present", scopeClaimName, scope)); + throw new JwtValidatorException(String.format("%s value must not contain duplicates - %s already present", scopeClaimName, scope)); copy.add(scope); } @@ -84,15 +86,15 @@ public class ClaimValidationUtils { * * @return Input parameter, as provided * - * @throws ValidateException Thrown if the value is null or negative + * @throws JwtValidatorException Thrown if the value is null or negative */ - public static long validateExpiration(String claimName, Long claimValue) throws ValidateException { + public static long validateExpiration(String claimName, Long claimValue) throws JwtValidatorException { if (claimValue == null) - throw new ValidateException(String.format("%s value must be non-null", claimName)); + throw new JwtValidatorException(String.format("%s value must be non-null", claimName)); if (claimValue < 0) - throw new ValidateException(String.format("%s value must be non-negative; value given was \"%s\"", claimName, claimValue)); + throw new JwtValidatorException(String.format("%s value must be non-negative; value given was \"%s\"", claimName, claimValue)); return claimValue; } @@ -112,10 +114,10 @@ public class ClaimValidationUtils { * * @return Trimmed version of the claimValue parameter * - * @throws ValidateException Thrown if the value is null, empty, or whitespace only + * @throws JwtValidatorException Thrown if the value is null, empty, or whitespace only */ - public static String validateSubject(String claimName, String claimValue) throws ValidateException { + public static String validateSubject(String claimName, String claimValue) throws JwtValidatorException { return validateString(claimName, claimValue); } @@ -132,12 +134,12 @@ public class ClaimValidationUtils { * * @return Input parameter, as provided * - * @throws ValidateException Thrown if the value is negative + * @throws JwtValidatorException Thrown if the value is negative */ - public static Long validateIssuedAt(String claimName, Long claimValue) throws ValidateException { + public static Long validateIssuedAt(String claimName, Long claimValue) throws JwtValidatorException { if (claimValue != null && claimValue < 0) - throw new ValidateException(String.format("%s value must be null or non-negative; value given was \"%s\"", claimName, claimValue)); + throw new JwtValidatorException(String.format("%s value must be null or non-negative; value given was \"%s\"", claimName, claimValue)); return claimValue; } @@ -157,24 +159,24 @@ public class ClaimValidationUtils { * * @return Trimmed version of the value parameter * - * @throws ValidateException Thrown if the value is null, empty, or whitespace only + * @throws JwtValidatorException Thrown if the value is null, empty, or whitespace only */ - public static String validateClaimNameOverride(String name, String value) throws ValidateException { + public static String validateClaimNameOverride(String name, String value) throws JwtValidatorException { return validateString(name, value); } - private static String validateString(String name, String value) throws ValidateException { + private static String validateString(String name, String value) throws JwtValidatorException { if (value == null) - throw new ValidateException(String.format("%s value must be non-null", name)); + throw new JwtValidatorException(String.format("%s value must be non-null", name)); if (value.isEmpty()) - throw new ValidateException(String.format("%s value must be non-empty", name)); + throw new JwtValidatorException(String.format("%s value must be non-empty", name)); value = value.trim(); if (value.isEmpty()) - throw new ValidateException(String.format("%s value must not contain only whitespace", name)); + throw new JwtValidatorException(String.format("%s value must not contain only whitespace", name)); return value; } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientCredentialsRequestFormatter.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientCredentialsRequestFormatter.java new file mode 100644 index 00000000000..f1eaf99b9aa --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientCredentialsRequestFormatter.java @@ -0,0 +1,91 @@ +/* + * 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.kafka.common.security.oauthbearer.internals.secured; + +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.utils.Utils; + +import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; +import java.util.Base64; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_ID; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_SECRET; + +public class ClientCredentialsRequestFormatter implements HttpRequestFormatter { + + public static final String GRANT_TYPE = "client_credentials"; + + private final String clientId; + + private final String clientSecret; + + private final String scope; + + public ClientCredentialsRequestFormatter(String clientId, String clientSecret, String scope, boolean urlencode) { + if (Utils.isBlank(clientId)) + throw new ConfigException(SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_ID, clientId); + + if (Utils.isBlank(clientSecret)) + throw new ConfigException(SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_SECRET, clientId); + + clientId = clientId.trim(); + clientSecret = clientSecret.trim(); + scope = Utils.isBlank(scope) ? null : scope.trim(); + + // according to RFC-6749 clientId & clientSecret must be urlencoded, see https://tools.ietf.org/html/rfc6749#section-2.3.1 + if (urlencode) { + clientId = URLEncoder.encode(clientId, StandardCharsets.UTF_8); + clientSecret = URLEncoder.encode(clientSecret, StandardCharsets.UTF_8); + + if (scope != null) + scope = URLEncoder.encode(scope, StandardCharsets.UTF_8); + } + + this.clientId = clientId; + this.clientSecret = clientSecret; + this.scope = scope; + } + + @Override + public Map formatHeaders() { + String s = String.format("%s:%s", clientId, clientSecret); + // Per RFC-7617, we need to use the *non-URL safe* base64 encoder. See KAFKA-14496. + String encoded = Base64.getEncoder().encodeToString(Utils.utf8(s)); + String authorizationHeader = String.format("Basic %s", encoded); + + Map headers = new HashMap<>(); + headers.put("Accept", "application/json"); + headers.put("Authorization", authorizationHeader); + headers.put("Cache-Control", "no-cache"); + headers.put("Content-Type", "application/x-www-form-urlencoded"); + return headers; + } + + @Override + public String formatBody() { + StringBuilder requestParameters = new StringBuilder(); + requestParameters.append("grant_type=").append(GRANT_TYPE); + + if (scope != null) + requestParameters.append("&scope=").append(scope); + + return requestParameters.toString(); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CloseableVerificationKeyResolver.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CloseableVerificationKeyResolver.java index bf8ca0cb822..d38d0708e94 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CloseableVerificationKeyResolver.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CloseableVerificationKeyResolver.java @@ -21,33 +21,14 @@ import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback import org.jose4j.keys.resolvers.VerificationKeyResolver; -import java.io.Closeable; -import java.io.IOException; - /** * The {@link OAuthBearerValidatorCallbackHandler} uses a {@link VerificationKeyResolver} as * part of its validation of the incoming JWT. Some of the VerificationKeyResolver * implementations use resources like threads, connections, etc. that should be properly closed * when no longer needed. Since the VerificationKeyResolver interface itself doesn't * define a close method, we provide a means to do that here. - * - * @see OAuthBearerValidatorCallbackHandler - * @see VerificationKeyResolver - * @see Closeable */ -public interface CloseableVerificationKeyResolver extends Initable, Closeable, VerificationKeyResolver { - - /** - * Lifecycle method to perform a clean shutdown of the {@link VerificationKeyResolver}. - * This must be performed by the caller to ensure the correct state, freeing up - * and releasing any resources performed in {@link #init()}. - * - * @throws IOException Thrown on errors related to IO during closure - */ - - default void close() throws IOException { - // This method left intentionally blank. - } +public interface CloseableVerificationKeyResolver extends OAuthBearerConfigurable, VerificationKeyResolver { } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtils.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtils.java index 10f700826c8..a0819766a38 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtils.java @@ -18,19 +18,25 @@ package org.apache.kafka.common.security.oauthbearer.internals.secured; import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.config.types.Password; import org.apache.kafka.common.network.ListenerName; +import org.apache.kafka.common.utils.Utils; import java.io.File; import java.net.MalformedURLException; import java.net.URISyntaxException; import java.net.URL; -import java.nio.file.Path; import java.util.Arrays; +import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +import javax.security.auth.login.AppConfigurationEntry; + +import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_FILES_CONFIG; +import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_FILES_DEFAULT; import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG; import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_DEFAULT; @@ -58,6 +64,10 @@ public class ConfigurationUtils { this.prefix = null; } + public boolean containsKey(String name) { + return get(name) != null; + } + /** * Validates that, if a value is supplied, is a file that: * @@ -71,7 +81,7 @@ public class ConfigurationUtils { * ignored. Any whitespace is trimmed off of the beginning and end. */ - public Path validateFile(String name) { + public File validateFileUrl(String name) { URL url = validateUrl(name); File file; @@ -81,6 +91,35 @@ public class ConfigurationUtils { throw new ConfigException(String.format("The OAuth configuration option %s contains a URL (%s) that is malformed: %s", name, url, e.getMessage())); } + return validateFile(name, file); + } + + /** + * Validates that the file: + * + *

  • + *
      exists
    + *
      has read permission
    + *
      points to a file
    + *
  • + */ + public File validateFile(String name) { + String s = validateString(name); + File file = validateFile(name, new File(s).getAbsoluteFile()); + throwIfFileIsNotAllowed(name, file.getAbsolutePath()); + return file; + } + + /** + * Validates that the file: + * + *
  • + *
      exists
    + *
      has read permission
    + *
      points to a file
    + *
  • + */ + private File validateFile(String name, File file) { if (!file.exists()) throw new ConfigException(String.format("The OAuth configuration option %s contains a file (%s) that doesn't exist", name, file)); @@ -90,7 +129,7 @@ public class ConfigurationUtils { if (file.isDirectory()) throw new ConfigException(String.format("The OAuth configuration option %s references a directory (%s), not a file", name, file)); - return file.toPath(); + return file; } /** @@ -110,7 +149,7 @@ public class ConfigurationUtils { if (value == null) { if (isRequired) - throw new ConfigException(String.format("The OAuth configuration option %s must be non-null", name)); + throw new ConfigException(String.format("The OAuth configuration option %s is required", name)); else return null; } @@ -143,7 +182,7 @@ public class ConfigurationUtils { if (value == null) { if (isRequired) - throw new ConfigException(String.format("The OAuth configuration option %s must be non-null", name)); + throw new ConfigException(String.format("The OAuth configuration option %s is required", name)); else return null; } @@ -187,42 +226,42 @@ public class ConfigurationUtils { if (!(protocol.equals("http") || protocol.equals("https") || protocol.equals("file"))) throw new ConfigException(String.format("The OAuth configuration option %s contains a URL (%s) that contains an invalid protocol (%s); only \"http\", \"https\", and \"file\" protocol are supported", name, value, protocol)); - throwIfURLIsNotAllowed(value); + throwIfURLIsNotAllowed(name, value); return url; } - public String validateString(String name) throws ValidateException { + public String validatePassword(String name) { + Password value = get(name); + + if (value == null || Utils.isBlank(value.value())) + throw new ConfigException(String.format("The OAuth configuration option %s value is required", name)); + + return value.value().trim(); + } + + public String validateString(String name) { return validateString(name, true); } - public String validateString(String name, boolean isRequired) throws ValidateException { + public String validateString(String name, boolean isRequired) { String value = get(name); - if (value == null) { + if (Utils.isBlank(value)) { if (isRequired) - throw new ConfigException(String.format("The OAuth configuration option %s value must be non-null", name)); + throw new ConfigException(String.format("The OAuth configuration option %s value is required", name)); else return null; } - value = value.trim(); - - if (value.isEmpty()) { - if (isRequired) - throw new ConfigException(String.format("The OAuth configuration option %s value must not contain only whitespace", name)); - else - return null; - } - - return value; + return value.trim(); } public Boolean validateBoolean(String name, boolean isRequired) { Boolean value = get(name); if (value == null && isRequired) - throw new ConfigException(String.format("The OAuth configuration option %s must be non-null", name)); + throw new ConfigException(String.format("The OAuth configuration option %s is required", name)); return value; } @@ -237,16 +276,130 @@ public class ConfigurationUtils { return (T) configs.get(name); } + public static T getConfiguredInstance(Map configs, + String saslMechanism, + List jaasConfigEntries, + String configName, + Class expectedClass) { + Object configValue = configs.get(configName); + Object o; + + if (configValue instanceof String) { + String implementationClassName = (String) configValue; + + try { + o = Utils.newInstance(implementationClassName, expectedClass); + } catch (Exception e) { + throw new ConfigException( + String.format( + "The class %s defined in the %s configuration could not be instantiated: %s", + implementationClassName, + configName, + e.getMessage() + ) + ); + } + } else if (configValue instanceof Class) { + Class implementationClass = (Class) configValue; + + try { + o = Utils.newInstance(implementationClass); + } catch (Exception e) { + throw new ConfigException( + String.format( + "The class %s defined in the %s configuration could not be instantiated: %s", + implementationClass.getName(), + configName, + e.getMessage() + ) + ); + } + } else if (configValue != null) { + throw new ConfigException( + String.format( + "The type for the %s configuration must be either %s or %s, but was %s", + configName, + String.class.getName(), + Class.class.getName(), + configValue.getClass().getName() + ) + ); + } else { + throw new ConfigException(String.format("The required configuration %s was null", configName)); + } + + if (!expectedClass.isInstance(o)) { + throw new ConfigException( + String.format( + "The configured class (%s) for the %s configuration is not an instance of %s, as is required", + o.getClass().getName(), + configName, + expectedClass.getName() + ) + ); + } + + if (o instanceof OAuthBearerConfigurable) { + try { + ((OAuthBearerConfigurable) o).configure(configs, saslMechanism, jaasConfigEntries); + } catch (Exception e) { + Utils.maybeCloseQuietly(o, "Instance of class " + o.getClass().getName() + " failed call to configure()"); + throw new ConfigException( + String.format( + "The class %s defined in the %s configuration encountered an error on configure(): %s", + o.getClass().getName(), + configName, + e.getMessage() + ) + ); + } + } + + return expectedClass.cast(o); + } + // visible for testing // make sure the url is in the "org.apache.kafka.sasl.oauthbearer.allowed.urls" system property - void throwIfURLIsNotAllowed(String value) { - Set allowedUrls = Arrays.stream( - System.getProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, ALLOWED_SASL_OAUTHBEARER_URLS_DEFAULT).split(",")) - .map(String::trim) - .collect(Collectors.toSet()); - if (!allowedUrls.contains(value)) { - throw new ConfigException(value + " is not allowed. Update system property '" - + ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG + "' to allow " + value); + void throwIfURLIsNotAllowed(String configName, String configValue) { + throwIfResourceIsNotAllowed( + "URL", + configName, + configValue, + ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, + ALLOWED_SASL_OAUTHBEARER_URLS_DEFAULT + ); + } + + // visible for testing + // make sure the file is in the "org.apache.kafka.sasl.oauthbearer.allowed.files" system property + void throwIfFileIsNotAllowed(String configName, String configValue) { + throwIfResourceIsNotAllowed( + "file", + configName, + configValue, + ALLOWED_SASL_OAUTHBEARER_FILES_CONFIG, + ALLOWED_SASL_OAUTHBEARER_FILES_DEFAULT + ); + } + + private void throwIfResourceIsNotAllowed(String resourceType, + String configName, + String configValue, + String propertyName, + String propertyDefault) { + String[] allowedArray = System.getProperty(propertyName, propertyDefault).split(","); + Set allowed = Arrays.stream(allowedArray) + .map(String::trim) + .collect(Collectors.toSet()); + + if (!allowed.contains(configValue)) { + String message = String.format( + "The %s cannot be accessed due to restrictions. Update the system property '%s' to allow the %s to be accessed.", + resourceType, + propertyName, + resourceType + ); + throw new ConfigException(configName, configValue, message); } } } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetriever.java deleted file mode 100644 index 2d607ddcda8..00000000000 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetriever.java +++ /dev/null @@ -1,131 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.common.security.oauthbearer.internals.secured; - -import org.apache.kafka.common.config.SaslConfigs; -import org.apache.kafka.common.utils.Utils; - -import java.io.IOException; -import java.net.URL; -import java.util.Locale; -import java.util.Map; - -import javax.net.ssl.SSLSocketFactory; - -import static org.apache.kafka.common.config.SaslConfigs.DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE; -import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS; -import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS; -import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS; -import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS; -import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_HEADER_URLENCODE; -import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL; -import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_ID_CONFIG; -import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_SECRET_CONFIG; -import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.SCOPE_CONFIG; - -/** - * {@code DefaultJwtRetriever} instantiates and delegates {@link JwtRetriever} API calls to an embedded implementation - * based on configuration. If {@link SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL} is configured with a - * {@code file}-based URL, a {@link FileJwtRetriever} is created and the JWT is expected be contained in the file - * specified. Otherwise, it's assumed to be an HTTP/HTTPS-based URL, so an {@link HttpJwtRetriever} is created. - */ -public class DefaultJwtRetriever implements JwtRetriever { - - private final Map configs; - private final String saslMechanism; - private final Map jaasConfig; - - private JwtRetriever delegate; - - public DefaultJwtRetriever(Map configs, String saslMechanism, Map jaasConfig) { - this.configs = configs; - this.saslMechanism = saslMechanism; - this.jaasConfig = jaasConfig; - } - - @Override - public void init() throws IOException { - ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism); - URL tokenEndpointUrl = cu.validateUrl(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL); - - if (tokenEndpointUrl.getProtocol().toLowerCase(Locale.ROOT).equals("file")) { - delegate = new FileJwtRetriever(cu.validateFile(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL)); - } else { - JaasOptionsUtils jou = new JaasOptionsUtils(jaasConfig); - String clientId = jou.validateString(CLIENT_ID_CONFIG); - String clientSecret = jou.validateString(CLIENT_SECRET_CONFIG); - String scope = jou.validateString(SCOPE_CONFIG, false); - - SSLSocketFactory sslSocketFactory = null; - - if (jou.shouldCreateSSLSocketFactory(tokenEndpointUrl)) - sslSocketFactory = jou.createSSLSocketFactory(); - - boolean urlencodeHeader = validateUrlencodeHeader(cu); - - delegate = new HttpJwtRetriever(clientId, - clientSecret, - scope, - sslSocketFactory, - tokenEndpointUrl.toString(), - cu.validateLong(SASL_LOGIN_RETRY_BACKOFF_MS), - cu.validateLong(SASL_LOGIN_RETRY_BACKOFF_MAX_MS), - cu.validateInteger(SASL_LOGIN_CONNECT_TIMEOUT_MS, false), - cu.validateInteger(SASL_LOGIN_READ_TIMEOUT_MS, false), - urlencodeHeader); - } - - delegate.init(); - } - - @Override - public String retrieve() throws IOException { - if (delegate == null) - throw new IllegalStateException("JWT retriever delegate is null; please call init() first"); - - return delegate.retrieve(); - } - - @Override - public void close() throws IOException { - Utils.closeQuietly(delegate, "JWT retriever delegate"); - } - - /** - * In some cases, the incoming {@link Map} doesn't contain a value for - * {@link SaslConfigs#SASL_OAUTHBEARER_HEADER_URLENCODE}. Returning {@code null} from {@link Map#get(Object)} - * will cause a {@link NullPointerException} when it is later unboxed. - * - *

    - * - * This utility method ensures that we have a non-{@code null} value to use in the - * {@link HttpJwtRetriever} constructor. - */ - static boolean validateUrlencodeHeader(ConfigurationUtils configurationUtils) { - Boolean urlencodeHeader = configurationUtils.get(SASL_OAUTHBEARER_HEADER_URLENCODE); - - if (urlencodeHeader != null) - return urlencodeHeader; - else - return DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE; - } - - JwtRetriever delegate() { - return delegate; - } -} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java deleted file mode 100644 index 5cd1e61db88..00000000000 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java +++ /dev/null @@ -1,108 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.common.security.oauthbearer.internals.secured; - -import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken; -import org.apache.kafka.common.utils.Utils; - -import org.jose4j.keys.resolvers.VerificationKeyResolver; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; - -import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS; -import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE; -import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER; -import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME; -import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME; - -/** - * This {@link JwtValidator} uses the delegation approach, instantiating and delegating calls to a - * more concrete implementation. The underlying implementation is determined by the presence/absence - * of the {@link VerificationKeyResolver}: if it's present, a {@link BrokerJwtValidator} is - * created, otherwise a {@link ClientJwtValidator} is created. - */ -public class DefaultJwtValidator implements JwtValidator { - - private final Map configs; - private final String saslMechanism; - private final Optional verificationKeyResolver; - - private JwtValidator delegate; - - public DefaultJwtValidator(Map configs, String saslMechanism) { - this.configs = configs; - this.saslMechanism = saslMechanism; - this.verificationKeyResolver = Optional.empty(); - } - - public DefaultJwtValidator(Map configs, - String saslMechanism, - VerificationKeyResolver verificationKeyResolver) { - this.configs = configs; - this.saslMechanism = saslMechanism; - this.verificationKeyResolver = Optional.of(verificationKeyResolver); - } - - @Override - public void init() throws IOException { - ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism); - - if (verificationKeyResolver.isPresent()) { - List expectedAudiencesList = cu.get(SASL_OAUTHBEARER_EXPECTED_AUDIENCE); - Set expectedAudiences = expectedAudiencesList != null ? Set.copyOf(expectedAudiencesList) : null; - Integer clockSkew = cu.validateInteger(SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS, false); - String expectedIssuer = cu.validateString(SASL_OAUTHBEARER_EXPECTED_ISSUER, false); - String scopeClaimName = cu.validateString(SASL_OAUTHBEARER_SCOPE_CLAIM_NAME); - String subClaimName = cu.validateString(SASL_OAUTHBEARER_SUB_CLAIM_NAME); - - delegate = new BrokerJwtValidator(clockSkew, - expectedAudiences, - expectedIssuer, - verificationKeyResolver.get(), - scopeClaimName, - subClaimName); - } else { - String scopeClaimName = cu.get(SASL_OAUTHBEARER_SCOPE_CLAIM_NAME); - String subClaimName = cu.get(SASL_OAUTHBEARER_SUB_CLAIM_NAME); - delegate = new ClientJwtValidator(scopeClaimName, subClaimName); - } - - delegate.init(); - } - - @Override - public OAuthBearerToken validate(String accessToken) throws ValidateException { - if (delegate == null) - throw new IllegalStateException("JWT validator delegate is null; please call init() first"); - - return delegate.validate(accessToken); - } - - @Override - public void close() throws IOException { - Utils.closeQuietly(delegate, "JWT validator delegate"); - } - - JwtValidator delegate() { - return delegate; - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileJwtRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileJwtRetriever.java deleted file mode 100644 index f04b5600168..00000000000 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileJwtRetriever.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.common.security.oauthbearer.internals.secured; - -import org.apache.kafka.common.utils.Utils; - -import java.io.IOException; -import java.nio.file.Path; - -/** - * FileJwtRetriever is an {@link JwtRetriever} that will load the contents - * of a file, interpreting them as a JWT access key in the serialized form. - * - * @see JwtRetriever - */ - -public class FileJwtRetriever implements JwtRetriever { - - private final Path accessTokenFile; - - private String accessToken; - - public FileJwtRetriever(Path accessTokenFile) { - this.accessTokenFile = accessTokenFile; - } - - @Override - public void init() throws IOException { - this.accessToken = Utils.readFileAsString(accessTokenFile.toFile().getPath()); - // always non-null; to remove any newline chars or backend will report err - this.accessToken = this.accessToken.trim(); - } - - @Override - public String retrieve() throws IOException { - if (accessToken == null) - throw new IllegalStateException("Access token is null; please call init() first"); - - return accessToken; - } - -} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpJwtRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpJwtRetriever.java index 35d25564bc0..4ae838e1f28 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpJwtRetriever.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpJwtRetriever.java @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.oauthbearer.internals.secured; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.security.oauthbearer.JwtRetriever; +import org.apache.kafka.common.security.oauthbearer.JwtRetrieverException; import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler; -import org.apache.kafka.common.utils.Utils; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; @@ -35,11 +35,9 @@ import java.io.InputStream; import java.io.OutputStream; import java.net.HttpURLConnection; import java.net.URL; -import java.net.URLEncoder; import java.nio.charset.StandardCharsets; -import java.util.Base64; -import java.util.Collections; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; @@ -47,6 +45,13 @@ import java.util.concurrent.ExecutionException; import javax.net.ssl.HttpsURLConnection; import javax.net.ssl.SSLSocketFactory; +import javax.security.auth.login.AppConfigurationEntry; + +import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS; +import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS; +import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS; +import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL; /** * HttpJwtRetriever is a {@link JwtRetriever} that will communicate with an OAuth/OIDC @@ -60,10 +65,6 @@ public class HttpJwtRetriever implements JwtRetriever { private static final Set UNRETRYABLE_HTTP_CODES; - private static final int MAX_RESPONSE_BODY_LENGTH = 1000; - - public static final String AUTHORIZATION_HEADER = "Authorization"; - static { // This does not have to be an exhaustive list. There are other HTTP codes that // are defined in different RFCs (e.g. https://datatracker.ietf.org/doc/html/rfc6585) @@ -89,46 +90,38 @@ public class HttpJwtRetriever implements JwtRetriever { UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_VERSION); } - private final String clientId; + private final HttpRequestFormatter requestFormatter; - private final String clientSecret; + private SSLSocketFactory sslSocketFactory; - private final String scope; + private URL tokenEndpointUrl; - private final SSLSocketFactory sslSocketFactory; + private long loginRetryBackoffMs; - private final String tokenEndpointUrl; + private long loginRetryBackoffMaxMs; - private final long loginRetryBackoffMs; + private Integer loginConnectTimeoutMs; - private final long loginRetryBackoffMaxMs; + private Integer loginReadTimeoutMs; - private final Integer loginConnectTimeoutMs; + public HttpJwtRetriever(HttpRequestFormatter requestFormatter) { + this.requestFormatter = Objects.requireNonNull(requestFormatter); + } - private final Integer loginReadTimeoutMs; + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism); + JaasOptionsUtils jou = new JaasOptionsUtils(saslMechanism, jaasConfigEntries); - private final boolean urlencodeHeader; + tokenEndpointUrl = cu.validateUrl(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL); - public HttpJwtRetriever(String clientId, - String clientSecret, - String scope, - SSLSocketFactory sslSocketFactory, - String tokenEndpointUrl, - long loginRetryBackoffMs, - long loginRetryBackoffMaxMs, - Integer loginConnectTimeoutMs, - Integer loginReadTimeoutMs, - boolean urlencodeHeader) { - this.clientId = Objects.requireNonNull(clientId); - this.clientSecret = Objects.requireNonNull(clientSecret); - this.scope = scope; - this.sslSocketFactory = sslSocketFactory; - this.tokenEndpointUrl = Objects.requireNonNull(tokenEndpointUrl); - this.loginRetryBackoffMs = loginRetryBackoffMs; - this.loginRetryBackoffMaxMs = loginRetryBackoffMaxMs; - this.loginConnectTimeoutMs = loginConnectTimeoutMs; - this.loginReadTimeoutMs = loginReadTimeoutMs; - this.urlencodeHeader = urlencodeHeader; + if (jou.shouldCreateSSLSocketFactory(tokenEndpointUrl)) + sslSocketFactory = jou.createSSLSocketFactory(); + + this.loginRetryBackoffMs = cu.validateLong(SASL_LOGIN_RETRY_BACKOFF_MS); + this.loginRetryBackoffMaxMs = cu.validateLong(SASL_LOGIN_RETRY_BACKOFF_MAX_MS); + this.loginConnectTimeoutMs = cu.validateInteger(SASL_LOGIN_CONNECT_TIMEOUT_MS, false); + this.loginReadTimeoutMs = cu.validateInteger(SASL_LOGIN_READ_TIMEOUT_MS, false); } /** @@ -143,15 +136,12 @@ public class HttpJwtRetriever implements JwtRetriever { * * @return Non-null JWT access token string * - * @throws IOException Thrown on errors related to IO during retrieval + * @throws JwtRetrieverException Thrown on errors related to IO, parsing, etc. during retrieval */ - - @Override - public String retrieve() throws IOException { - String authorizationHeader = formatAuthorizationHeader(clientId, clientSecret, urlencodeHeader); - String requestBody = formatRequestBody(scope); + public String retrieve() throws JwtRetrieverException { + String requestBody = requestFormatter.formatBody(); Retry retry = new Retry<>(loginRetryBackoffMs, loginRetryBackoffMaxMs); - Map headers = Collections.singletonMap(AUTHORIZATION_HEADER, authorizationHeader); + Map headers = requestFormatter.formatHeaders(); String responseBody; @@ -160,7 +150,7 @@ public class HttpJwtRetriever implements JwtRetriever { HttpURLConnection con = null; try { - con = (HttpURLConnection) new URL(tokenEndpointUrl).openConnection(); + con = (HttpURLConnection) tokenEndpointUrl.openConnection(); if (sslSocketFactory != null && con instanceof HttpsURLConnection) ((HttpsURLConnection) con).setSSLSocketFactory(sslSocketFactory); @@ -174,13 +164,14 @@ public class HttpJwtRetriever implements JwtRetriever { } }); } catch (ExecutionException e) { - if (e.getCause() instanceof IOException) - throw (IOException) e.getCause(); + if (e.getCause() instanceof JwtRetrieverException) + throw (JwtRetrieverException) e.getCause(); else throw new KafkaException(e.getCause()); } - return parseAccessToken(responseBody); + JwtResponseParser responseParser = new JwtResponseParser(); + return responseParser.parseJwt(responseBody); } public static String post(HttpURLConnection con, @@ -322,71 +313,4 @@ public class HttpJwtRetriever implements JwtRetriever { } return String.format("{%s}", errorResponseBody); } - - static String parseAccessToken(String responseBody) throws IOException { - ObjectMapper mapper = new ObjectMapper(); - JsonNode rootNode = mapper.readTree(responseBody); - JsonNode accessTokenNode = rootNode.at("/access_token"); - - if (accessTokenNode == null) { - // Only grab the first N characters so that if the response body is huge, we don't - // blow up. - String snippet = responseBody; - - if (snippet.length() > MAX_RESPONSE_BODY_LENGTH) { - int actualLength = responseBody.length(); - String s = responseBody.substring(0, MAX_RESPONSE_BODY_LENGTH); - snippet = String.format("%s (trimmed to first %d characters out of %d total)", s, MAX_RESPONSE_BODY_LENGTH, actualLength); - } - - throw new IOException(String.format("The token endpoint response did not contain an access_token value. Response: (%s)", snippet)); - } - - return sanitizeString("the token endpoint response's access_token JSON attribute", accessTokenNode.textValue()); - } - - static String formatAuthorizationHeader(String clientId, String clientSecret, boolean urlencode) { - clientId = sanitizeString("the token endpoint request client ID parameter", clientId); - clientSecret = sanitizeString("the token endpoint request client secret parameter", clientSecret); - - // according to RFC-6749 clientId & clientSecret must be urlencoded, see https://tools.ietf.org/html/rfc6749#section-2.3.1 - if (urlencode) { - clientId = URLEncoder.encode(clientId, StandardCharsets.UTF_8); - clientSecret = URLEncoder.encode(clientSecret, StandardCharsets.UTF_8); - } - - String s = String.format("%s:%s", clientId, clientSecret); - // Per RFC-7617, we need to use the *non-URL safe* base64 encoder. See KAFKA-14496. - String encoded = Base64.getEncoder().encodeToString(Utils.utf8(s)); - return String.format("Basic %s", encoded); - } - - static String formatRequestBody(String scope) { - StringBuilder requestParameters = new StringBuilder(); - requestParameters.append("grant_type=client_credentials"); - - if (scope != null && !scope.trim().isEmpty()) { - scope = scope.trim(); - String encodedScope = URLEncoder.encode(scope, StandardCharsets.UTF_8); - requestParameters.append("&scope=").append(encodedScope); - } - - return requestParameters.toString(); - } - - private static String sanitizeString(String name, String value) { - if (value == null) - throw new IllegalArgumentException(String.format("The value for %s must be non-null", name)); - - if (value.isEmpty()) - throw new IllegalArgumentException(String.format("The value for %s must be non-empty", name)); - - value = value.trim(); - - if (value.isEmpty()) - throw new IllegalArgumentException(String.format("The value for %s must not contain only whitespace", name)); - - return value; - } - } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Initable.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpRequestFormatter.java similarity index 66% rename from clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Initable.java rename to clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpRequestFormatter.java index eff1b543886..a1a63603a61 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Initable.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpRequestFormatter.java @@ -14,21 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.oauthbearer.internals.secured; -import java.io.IOException; +import java.util.Map; -public interface Initable { +public interface HttpRequestFormatter { - /** - * Lifecycle method to perform any one-time initialization of a given resource. This must - * be invoked by the caller to ensure the correct state before methods are invoked. - * - * @throws IOException Thrown on errors related to IO during initialization - */ + Map formatHeaders(); - default void init() throws IOException { - // This method left intentionally blank. - } + String formatBody(); } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JaasOptionsUtils.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JaasOptionsUtils.java index 3e49595dbc1..ec6d3daafe8 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JaasOptionsUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JaasOptionsUtils.java @@ -20,10 +20,12 @@ package org.apache.kafka.common.security.oauthbearer.internals.secured; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.config.types.Password; import org.apache.kafka.common.network.ConnectionMode; import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule; import org.apache.kafka.common.security.ssl.DefaultSslEngineFactory; import org.apache.kafka.common.security.ssl.SslFactory; +import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,6 +54,10 @@ public class JaasOptionsUtils { this.options = options; } + public JaasOptionsUtils(String saslMechanism, List jaasConfigEntries) { + this.options = getOptions(saslMechanism, jaasConfigEntries); + } + public static Map getOptions(String saslMechanism, List jaasConfigEntries) { if (!OAuthBearerLoginModule.OAUTHBEARER_MECHANISM.equals(saslMechanism)) throw new IllegalArgumentException(String.format("Unexpected SASL mechanism: %s", saslMechanism)); @@ -62,6 +68,10 @@ public class JaasOptionsUtils { return Collections.unmodifiableMap(jaasConfigEntries.get(0).getOptions()); } + public boolean containsKey(String name) { + return options.containsKey(name); + } + public boolean shouldCreateSSLSocketFactory(URL url) { return url.getProtocol().equalsIgnoreCase("https"); } @@ -82,30 +92,29 @@ public class JaasOptionsUtils { return socketFactory; } - public String validateString(String name) throws ValidateException { + public String validatePassword(String name) { + Password value = (Password) options.get(name); + + if (value == null || Utils.isBlank(value.value())) + throw new ConfigException(String.format("The OAuth configuration option %s value is required", name)); + + return value.value().trim(); + } + + public String validateString(String name) { return validateString(name, true); } - public String validateString(String name, boolean isRequired) throws ValidateException { + public String validateString(String name, boolean isRequired) { String value = (String) options.get(name); - if (value == null) { + if (Utils.isBlank(value)) { if (isRequired) - throw new ConfigException(String.format("The OAuth configuration option %s value must be non-null", name)); + throw new ConfigException(String.format("The OAuth configuration option %s value is required", name)); else return null; } - value = value.trim(); - - if (value.isEmpty()) { - if (isRequired) - throw new ConfigException(String.format("The OAuth configuration option %s value must not contain only whitespace", name)); - else - return null; - } - - return value; + return value.trim(); } - } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwksFileVerificationKeyResolver.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwksFileVerificationKeyResolver.java index 27cdccb286c..170b0271a09 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwksFileVerificationKeyResolver.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwksFileVerificationKeyResolver.java @@ -17,22 +17,26 @@ package org.apache.kafka.common.security.oauthbearer.internals.secured; -import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.common.config.ConfigException; import org.jose4j.jwk.JsonWebKeySet; import org.jose4j.jws.JsonWebSignature; import org.jose4j.jwx.JsonWebStructure; import org.jose4j.keys.resolvers.JwksVerificationKeyResolver; import org.jose4j.keys.resolvers.VerificationKeyResolver; -import org.jose4j.lang.JoseException; import org.jose4j.lang.UnresolvableKeyException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.nio.file.Path; +import java.io.File; import java.security.Key; import java.util.List; +import java.util.Map; + +import javax.security.auth.login.AppConfigurationEntry; + +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.CachedFile.RefreshPolicy.lastModifiedPolicy; /** * JwksFileVerificationKeyResolver is a {@link VerificationKeyResolver} implementation @@ -79,41 +83,46 @@ import java.util.List; * @see org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL * @see VerificationKeyResolver */ - public class JwksFileVerificationKeyResolver implements CloseableVerificationKeyResolver { private static final Logger log = LoggerFactory.getLogger(JwksFileVerificationKeyResolver.class); - private final Path jwksFile; - - private VerificationKeyResolver delegate; - - public JwksFileVerificationKeyResolver(Path jwksFile) { - this.jwksFile = jwksFile; - } + private CachedFile delegate; @Override - public void init() throws IOException { - log.debug("Starting creation of new VerificationKeyResolver from {}", jwksFile); - String json = Utils.readFileAsString(jwksFile.toFile().getPath()); - - JsonWebKeySet jwks; - - try { - jwks = new JsonWebKeySet(json); - } catch (JoseException e) { - throw new IOException(e); - } - - delegate = new JwksVerificationKeyResolver(jwks.getJsonWebKeys()); + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism); + File file = cu.validateFileUrl(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL); + delegate = new CachedFile<>(file, new VerificationKeyResolverTransformer(), lastModifiedPolicy()); } @Override public Key resolveKey(JsonWebSignature jws, List nestingContext) throws UnresolvableKeyException { if (delegate == null) - throw new UnresolvableKeyException("VerificationKeyResolver delegate is null; please call init() first"); + throw new UnresolvableKeyException("VerificationKeyResolver delegate is null; please call configure() first"); - return delegate.resolveKey(jws, nestingContext); + return delegate.transformed().resolveKey(jws, nestingContext); } + /** + * "Transforms" the raw file contents into a {@link VerificationKeyResolver} that can be used to resolve + * the keys provided in the JWT. + */ + private static class VerificationKeyResolverTransformer implements CachedFile.Transformer { + + @Override + public VerificationKeyResolver transform(File file, String contents) { + log.debug("Starting creation of new VerificationKeyResolver from {}", file.getPath()); + + JsonWebKeySet jwks; + + try { + jwks = new JsonWebKeySet(contents); + } catch (Exception e) { + throw new ConfigException(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL, file.getPath(), e.getMessage()); + } + + return new JwksVerificationKeyResolver(jwks.getJsonWebKeys()); + } + } } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtBearerRequestFormatter.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtBearerRequestFormatter.java new file mode 100644 index 00000000000..495d1434d98 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtBearerRequestFormatter.java @@ -0,0 +1,61 @@ +/* + * 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.kafka.common.security.oauthbearer.internals.secured; + + +import org.apache.kafka.common.utils.Utils; + +import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.Map; +import java.util.function.Supplier; + +public class JwtBearerRequestFormatter implements HttpRequestFormatter { + + public static final String GRANT_TYPE = "urn:ietf:params:oauth:grant-type:jwt-bearer"; + + private final String scope; + private final Supplier assertionSupplier; + + public JwtBearerRequestFormatter(String scope, Supplier assertionSupplier) { + this.scope = scope; + this.assertionSupplier = assertionSupplier; + } + + @Override + public String formatBody() { + String assertion = assertionSupplier.get(); + StringBuilder requestParameters = new StringBuilder(); + requestParameters.append("grant_type=").append(URLEncoder.encode(GRANT_TYPE, StandardCharsets.UTF_8)); + requestParameters.append("&assertion=").append(URLEncoder.encode(assertion, StandardCharsets.UTF_8)); + + if (!Utils.isBlank(scope)) + requestParameters.append("&scope=").append(URLEncoder.encode(scope.trim(), StandardCharsets.UTF_8)); + + return requestParameters.toString(); + } + + @Override + public Map formatHeaders() { + Map headers = new HashMap<>(); + headers.put("Accept", "application/json"); + headers.put("Cache-Control", "no-cache"); + headers.put("Content-Type", "application/x-www-form-urlencoded"); + return headers; + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtResponseParser.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtResponseParser.java new file mode 100644 index 00000000000..bab996cd3e9 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtResponseParser.java @@ -0,0 +1,65 @@ +/* + * 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.kafka.common.security.oauthbearer.internals.secured; + +import org.apache.kafka.common.security.oauthbearer.JwtRetrieverException; +import org.apache.kafka.common.utils.Utils; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; + +import java.io.IOException; + +public class JwtResponseParser { + + private static final String[] JSON_PATHS = new String[] {"/access_token", "/id_token"}; + private static final int MAX_RESPONSE_BODY_LENGTH = 1000; + + public String parseJwt(String responseBody) throws JwtRetrieverException { + ObjectMapper mapper = new ObjectMapper(); + JsonNode rootNode; + + try { + rootNode = mapper.readTree(responseBody); + } catch (IOException e) { + throw new JwtRetrieverException(e); + } + + for (String jsonPath : JSON_PATHS) { + JsonNode node = rootNode.at(jsonPath); + + if (node != null && !node.isMissingNode()) { + String value = node.textValue(); + + if (!Utils.isBlank(value)) { + return value.trim(); + } + } + } + + // Only grab the first N characters so that if the response body is huge, we don't blow up. + String snippet = responseBody; + + if (snippet.length() > MAX_RESPONSE_BODY_LENGTH) { + int actualLength = responseBody.length(); + String s = responseBody.substring(0, MAX_RESPONSE_BODY_LENGTH); + snippet = String.format("%s (trimmed to first %d characters out of %d total)", s, MAX_RESPONSE_BODY_LENGTH, actualLength); + } + + throw new JwtRetrieverException(String.format("The token endpoint response did not contain a valid JWT. Response: (%s)", snippet)); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerConfigurable.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerConfigurable.java new file mode 100644 index 00000000000..4c721e17bff --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerConfigurable.java @@ -0,0 +1,88 @@ +/* + * 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.kafka.common.security.oauthbearer.internals.secured; + +import org.apache.kafka.common.Configurable; +import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler; + +import java.io.Closeable; +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.login.AppConfigurationEntry; + +/** + * Analogue to {@link Configurable} for OAuth-based authentication. This interface presents a similar + * method signature as that of the {@link AuthenticateCallbackHandler} interface. However, this interface is + * needed because {@link AuthenticateCallbackHandler} extends the JDK's {@link CallbackHandler} interface. + * + *

    + * + * Note: + * + *

      + *
    1. + * Any class that implements this interface should initialize resources via + * {@link #configure(Map, String, List)} and release them via {@link #close()}. + *
    2. + *
    3. + * Any class that instantiates an object that implements {@code OAuthBearerConfigurable} + * must properly call that object's ({@link #configure(Map, String, List)} and {@link #close()}) methods + * so that the object can initialize and release resources. + *
    4. + *
    + */ +public interface OAuthBearerConfigurable extends Closeable { + + /** + * Configures this object for the specified SASL mechanism. + * + * @param configs Key-value pairs containing the parsed configuration options of + * the client or broker. Note that these are the Kafka configuration options + * and not the JAAS configuration options. JAAS config options may be obtained + * from `jaasConfigEntries`. For configs that may be specified as both Kafka config + * as well as JAAS config (e.g. sasl.kerberos.service.name), the configuration + * is treated as invalid if conflicting values are provided. + * @param saslMechanism Negotiated SASL mechanism. For clients, this is the SASL + * mechanism configured for the client. For brokers, this is the mechanism + * negotiated with the client and is one of the mechanisms enabled on the broker. + * @param jaasConfigEntries JAAS configuration entries from the JAAS login context. + * This list contains a single entry for clients and may contain more than + * one entry for brokers if multiple mechanisms are enabled on a listener using + * static JAAS configuration where there is no mapping between mechanisms and + * login module entries. In this case, implementations can use the login module in + * `jaasConfigEntries` to identify the entry corresponding to `saslMechanism`. + * Alternatively, dynamic JAAS configuration option + * {@link org.apache.kafka.common.config.SaslConfigs#SASL_JAAS_CONFIG} may be + * configured on brokers with listener and mechanism prefix, in which case + * only the configuration entry corresponding to `saslMechanism` will be provided + * in `jaasConfigEntries`. + */ + default void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + + } + + /** + * Closes any resources that were initialized by {@link #configure(Map, String, List)}. + */ + default void close() throws IOException { + // Do nothing... + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwks.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwks.java index 4d75ff847ea..d8014010a7d 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwks.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwks.java @@ -17,6 +17,7 @@ package org.apache.kafka.common.security.oauthbearer.internals.secured; +import org.apache.kafka.common.security.oauthbearer.BrokerJwtValidator; import org.apache.kafka.common.utils.Time; import org.jose4j.jwk.HttpsJwks; @@ -25,7 +26,6 @@ import org.jose4j.lang.JoseException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.Closeable; import java.io.IOException; import java.util.Collections; import java.util.LinkedHashMap; @@ -56,8 +56,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; * @see org.jose4j.keys.resolvers.VerificationKeyResolver * @see BrokerJwtValidator */ - -public final class RefreshingHttpsJwks implements Initable, Closeable { +public final class RefreshingHttpsJwks implements OAuthBearerConfigurable { private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class); @@ -171,7 +170,6 @@ public final class RefreshingHttpsJwks implements Initable, Closeable { this(time, httpsJwks, refreshMs, refreshRetryBackoffMs, refreshRetryBackoffMaxMs, Executors.newSingleThreadScheduledExecutor()); } - @Override public void init() throws IOException { try { log.debug("init started"); @@ -375,5 +373,4 @@ public final class RefreshingHttpsJwks implements Initable, Closeable { } } } - } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwksVerificationKeyResolver.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwksVerificationKeyResolver.java index 52d0c6c3978..d6f6a010894 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwksVerificationKeyResolver.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwksVerificationKeyResolver.java @@ -17,6 +17,8 @@ package org.apache.kafka.common.security.oauthbearer.internals.secured; +import org.apache.kafka.common.KafkaException; + import org.jose4j.jwk.HttpsJwks; import org.jose4j.jwk.JsonWebKey; import org.jose4j.jwk.VerificationJwkSelector; @@ -31,6 +33,9 @@ import org.slf4j.LoggerFactory; import java.io.IOException; import java.security.Key; import java.util.List; +import java.util.Map; + +import javax.security.auth.login.AppConfigurationEntry; /** * RefreshingHttpsJwksVerificationKeyResolver is a @@ -80,7 +85,6 @@ import java.util.List; * @see RefreshingHttpsJwks * @see HttpsJwks */ - public class RefreshingHttpsJwksVerificationKeyResolver implements CloseableVerificationKeyResolver { private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwksVerificationKeyResolver.class); @@ -97,15 +101,14 @@ public class RefreshingHttpsJwksVerificationKeyResolver implements CloseableVeri } @Override - public void init() throws IOException { + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { try { - log.debug("init started"); - + log.debug("configure started"); refreshingHttpsJwks.init(); + } catch (IOException e) { + throw new KafkaException(e); } finally { isInitialized = true; - - log.debug("init completed"); } } @@ -123,7 +126,7 @@ public class RefreshingHttpsJwksVerificationKeyResolver implements CloseableVeri @Override public Key resolveKey(JsonWebSignature jws, List nestingContext) throws UnresolvableKeyException { if (!isInitialized) - throw new IllegalStateException("Please call init() first"); + throw new IllegalStateException("Please call configure() first"); try { List jwks = refreshingHttpsJwks.getJsonWebKeys(); @@ -148,5 +151,4 @@ public class RefreshingHttpsJwksVerificationKeyResolver implements CloseableVeri throw new UnresolvableKeyException(sb, e); } } - } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/SerializedJwt.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/SerializedJwt.java index f45865fa638..b9a50041096 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/SerializedJwt.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/SerializedJwt.java @@ -17,6 +17,8 @@ package org.apache.kafka.common.security.oauthbearer.internals.secured; +import org.apache.kafka.common.security.oauthbearer.JwtValidatorException; + /** * SerializedJwt provides a modicum of structure and validation around a JWT's serialized form by * splitting and making the three sections (header, payload, and signature) available to the user. @@ -39,12 +41,12 @@ public class SerializedJwt { token = token.trim(); if (token.isEmpty()) - throw new ValidateException("Malformed JWT provided; expected three sections (header, payload, and signature)"); + throw new JwtValidatorException("Malformed JWT provided; expected three sections (header, payload, and signature)"); String[] splits = token.split("\\."); if (splits.length != 3) - throw new ValidateException("Malformed JWT provided; expected three sections (header, payload, and signature)"); + throw new JwtValidatorException("Malformed JWT provided; expected three sections (header, payload, and signature)"); this.token = token.trim(); this.header = validateSection(splits[0]); @@ -92,11 +94,11 @@ public class SerializedJwt { return signature; } - private String validateSection(String section) throws ValidateException { + private String validateSection(String section) throws JwtValidatorException { section = section.trim(); if (section.isEmpty()) - throw new ValidateException("Malformed JWT provided; expected three sections (header, payload, and signature)"); + throw new JwtValidatorException("Malformed JWT provided; expected three sections (header, payload, and signature)"); return section; } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/VerificationKeyResolverFactory.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/VerificationKeyResolverFactory.java index c9ad41d5a97..85ad53246be 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/VerificationKeyResolverFactory.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/VerificationKeyResolverFactory.java @@ -17,55 +17,71 @@ package org.apache.kafka.common.security.oauthbearer.internals.secured; +import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler; import org.apache.kafka.common.utils.Time; import org.jose4j.http.Get; import org.jose4j.jwk.HttpsJwks; +import org.jose4j.jws.JsonWebSignature; +import org.jose4j.jwx.JsonWebStructure; +import org.jose4j.lang.UnresolvableKeyException; +import java.io.IOException; import java.net.URL; -import java.nio.file.Path; +import java.security.Key; +import java.util.HashMap; +import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicInteger; import javax.net.ssl.SSLSocketFactory; +import javax.security.auth.login.AppConfigurationEntry; import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS; import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS; import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS; import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL; +/** + * Because a {@link CloseableVerificationKeyResolver} instance can spawn threads and issue + * HTTP(S) calls ({@link RefreshingHttpsJwksVerificationKeyResolver}), we only want to create + * a new instance for each particular set of configuration. Because each set of configuration + * may have multiple instances, we want to reuse the single instance. + */ public class VerificationKeyResolverFactory { - /** - * Create a {@link JwtRetriever} from the given - * {@link org.apache.kafka.common.config.SaslConfigs}. - * - * Note: the returned CloseableVerificationKeyResolver is not - * initialized here and must be done by the caller. - * - * Primarily exposed here for unit testing. - * - * @param configs SASL configuration - * - * @return Non-null {@link CloseableVerificationKeyResolver} - */ - public static CloseableVerificationKeyResolver create(Map configs, - Map jaasConfig) { - return create(configs, null, jaasConfig); + private static final Map CACHE = new HashMap<>(); + + public static synchronized CloseableVerificationKeyResolver get(Map configs, + String saslMechanism, + List jaasConfigEntries) { + VerificationKeyResolverKey key = new VerificationKeyResolverKey(configs, saslMechanism, jaasConfigEntries); + + return CACHE.computeIfAbsent(key, k -> + new RefCountingVerificationKeyResolver( + create( + configs, + saslMechanism, + jaasConfigEntries + ) + ) + ); } - public static CloseableVerificationKeyResolver create(Map configs, - String saslMechanism, - Map jaasConfig) { + static CloseableVerificationKeyResolver create(Map configs, + String saslMechanism, + List jaasConfigEntries) { ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism); URL jwksEndpointUrl = cu.validateUrl(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL); + CloseableVerificationKeyResolver resolver; if (jwksEndpointUrl.getProtocol().toLowerCase(Locale.ROOT).equals("file")) { - Path p = cu.validateFile(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL); - return new JwksFileVerificationKeyResolver(p); + resolver = new JwksFileVerificationKeyResolver(); } else { long refreshIntervalMs = cu.validateLong(SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS, true, 0L); - JaasOptionsUtils jou = new JaasOptionsUtils(jaasConfig); + JaasOptionsUtils jou = new JaasOptionsUtils(saslMechanism, jaasConfigEntries); SSLSocketFactory sslSocketFactory = null; if (jou.shouldCreateSSLSocketFactory(jwksEndpointUrl)) @@ -85,8 +101,87 @@ public class VerificationKeyResolverFactory { refreshIntervalMs, cu.validateLong(SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS), cu.validateLong(SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS)); - return new RefreshingHttpsJwksVerificationKeyResolver(refreshingHttpsJwks); + resolver = new RefreshingHttpsJwksVerificationKeyResolver(refreshingHttpsJwks); + } + + resolver.configure(configs, saslMechanism, jaasConfigEntries); + return resolver; + } + + /** + * VkrKey is a simple structure which encapsulates the criteria for different + * sets of configuration. This will allow us to use this object as a key in a {@link Map} + * to keep a single instance per key. + */ + + private static class VerificationKeyResolverKey { + + private final Map configs; + + private final String saslMechanism; + + private final Map moduleOptions; + + public VerificationKeyResolverKey(Map configs, + String saslMechanism, + List jaasConfigEntries) { + this.configs = configs; + this.saslMechanism = saslMechanism; + this.moduleOptions = JaasOptionsUtils.getOptions(saslMechanism, jaasConfigEntries); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + VerificationKeyResolverKey that = (VerificationKeyResolverKey) o; + return configs.equals(that.configs) && saslMechanism.equals(that.saslMechanism) && moduleOptions.equals(that.moduleOptions); + } + + @Override + public int hashCode() { + return Objects.hash(configs, saslMechanism, moduleOptions); } } + /** + * RefCountingVerificationKeyResolver allows us to share a single + * {@link CloseableVerificationKeyResolver} instance between multiple + * {@link AuthenticateCallbackHandler} instances and perform the lifecycle methods the + * appropriate number of times. + */ + + private static class RefCountingVerificationKeyResolver implements CloseableVerificationKeyResolver { + + private final CloseableVerificationKeyResolver delegate; + + private final AtomicInteger count = new AtomicInteger(0); + + public RefCountingVerificationKeyResolver(CloseableVerificationKeyResolver delegate) { + this.delegate = delegate; + } + + @Override + public Key resolveKey(JsonWebSignature jws, List nestingContext) throws UnresolvableKeyException { + return delegate.resolveKey(jws, nestingContext); + } + + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + if (count.incrementAndGet() == 1) + delegate.configure(configs, saslMechanism, jaasConfigEntries); + } + + @Override + public void close() throws IOException { + if (count.decrementAndGet() == 0) + delegate.close(); + } + } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/AssertionCreator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/AssertionCreator.java new file mode 100644 index 00000000000..5c619c63693 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/AssertionCreator.java @@ -0,0 +1,96 @@ +/* + * 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.kafka.common.security.oauthbearer.internals.secured.assertion; + +import java.io.Closeable; +import java.io.IOException; +import java.security.GeneralSecurityException; + +/** + * {@code AssertionCreator} is used to create a client-signed OAuth assertion that can be used with different + * grant types. See RFC 7521 for specifics. + * + *

    + * + * The assertion creator has three main steps: + * + *

      + *
    1. Create the JWT header
    2. + *
    3. Create the JWT payload
    4. + *
    5. Sign
    6. + *
    + * + *

    + * + * Step 1 is to dynamically create the JWT header. The implementation may add whatever values it needs, but + * the {@code alg} (algorithm), {@code kid} (key ID), and {@code type} (type) are usually present. Here is + * an example of the JSON version of the JWT header: + * + *

    + * {
    + *   "kid": "9d82418e64e0541066637ca8592d459c",
    + *   "alg": RS256,
    + *   "typ": "JWT",
    + * }
    + * 
    + * + *

    + * + * Step 2 is to create the JWT payload from the claims provided to {@link #create(AssertionJwtTemplate)}. Depending on the + * implementation, other claims may be dynamically generated and added to the JWT payload. Or, some of the + * claims in the incoming map could be ignored or modified. Here's an example where the implementation has + * added the {@code iat} (initialized at) and {@code exp} (expires) claims: + * + *

    + * {
    + *   "iat": 1741121401,
    + *   "exp": 1741125001,
    + *   "sub": "some-service-account",
    + *   "aud": "my_audience",
    + *   "iss": "https://example.com",
    + *   "...": "...",
    + * }
    + * 
    + * + *

    + * + * Step 3 is to use the configured private key to sign the header and payload and serialize in the compact + * JWT format. The means by which the private key (if any) is made available for use is up to the + * implementation. The private key could be loaded from a file, downloaded from a trusted resource, + * embedded in the configuration, etc. + */ +public interface AssertionCreator extends Closeable { + + /** + * Creates and signs an OAuth assertion by converting the given claims into JWT and then signing it using + * the configured algorithm. + * + *

    + * + * @param template {@link AssertionJwtTemplate} with optional header and/or claims to include in the JWT + */ + String create(AssertionJwtTemplate template) throws GeneralSecurityException, IOException; + + /** + * Closes any resources used by this implementation. The default implementation of + * this method is a no op, for convenience to implementors. + */ + @Override + default void close() throws IOException { + // Do nothing... + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/AssertionJwtTemplate.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/AssertionJwtTemplate.java new file mode 100644 index 00000000000..ce6599c1b1d --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/AssertionJwtTemplate.java @@ -0,0 +1,73 @@ +/* + * 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.kafka.common.security.oauthbearer.internals.secured.assertion; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Map; + +/** + * {@code AssertionJwtTemplate} is used to provide values for use by {@link AssertionCreator}. + * The JWT header and/or payload used in the assertion likely requires headers and claims. Not all identity + * providers require the same set of headers and claims; some may require a given header or claim while + * other identity providers may prohibit it. In order to provide the most flexibility, the header + * values and claims that are to be included in the JWT can be added via a template. + * + *

    + * + * Both the {@link #header()} and {@link #payload()} APIs return a map of Objects. This because the + * JSON specification allow values to be one of the following "types": + * + *

    + * + * However, because the maps must be converted into JSON, it's important that any nested types use standard + * Java type equivalents (Map, List, String, Integer, Double, and Boolean) so that the JSON library will + * know how to serialize the entire object graph. + */ +public interface AssertionJwtTemplate extends Closeable { + + /** + * Returns a map containing zero or more header values. + * + * @return Values to include in the JWT header + */ + Map header(); + + /** + * Returns a map containing zero or more JWT payload claim values. + * + * @return Values to include in the JWT payload + */ + Map payload(); + + /** + * Closes any resources used by this implementation. The default implementation of + * this method is a no op, for convenience to implementors. + */ + @Override + default void close() throws IOException { + // Do nothing... + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/AssertionUtils.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/AssertionUtils.java new file mode 100644 index 00000000000..c4eed76e195 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/AssertionUtils.java @@ -0,0 +1,150 @@ +/* + * 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.kafka.common.security.oauthbearer.internals.secured.assertion; + +import org.apache.kafka.common.security.oauthbearer.internals.secured.ConfigurationUtils; +import org.apache.kafka.common.utils.Time; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.security.GeneralSecurityException; +import java.security.KeyFactory; +import java.security.NoSuchAlgorithmException; +import java.security.PrivateKey; +import java.security.Signature; +import java.security.spec.PKCS8EncodedKeySpec; +import java.util.ArrayList; +import java.util.Base64; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import javax.crypto.Cipher; +import javax.crypto.EncryptedPrivateKeyInfo; +import javax.crypto.SecretKey; +import javax.crypto.SecretKeyFactory; +import javax.crypto.spec.PBEKeySpec; + +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_ASSERTION_ALGORITHM; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_AUD; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_EXP_SECONDS; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_ISS; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_JTI_INCLUDE; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_NBF_SECONDS; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_SUB; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_ASSERTION_TEMPLATE_FILE; + +/** + * Set of utilities for the OAuth JWT assertion logic. + */ +public class AssertionUtils { + + public static final String TOKEN_SIGNING_ALGORITHM_RS256 = "RS256"; + public static final String TOKEN_SIGNING_ALGORITHM_ES256 = "ES256"; + + /** + * Inspired by {@code org.apache.kafka.common.security.ssl.DefaultSslEngineFactory.PemStore}, which is not + * visible to reuse directly. + */ + public static PrivateKey privateKey(byte[] privateKeyContents, + Optional passphrase) throws GeneralSecurityException, IOException { + PKCS8EncodedKeySpec keySpec; + + if (passphrase.isPresent()) { + EncryptedPrivateKeyInfo keyInfo = new EncryptedPrivateKeyInfo(privateKeyContents); + String algorithm = keyInfo.getAlgName(); + SecretKeyFactory secretKeyFactory = SecretKeyFactory.getInstance(algorithm); + SecretKey pbeKey = secretKeyFactory.generateSecret(new PBEKeySpec(passphrase.get().toCharArray())); + Cipher cipher = Cipher.getInstance(algorithm); + cipher.init(Cipher.DECRYPT_MODE, pbeKey, keyInfo.getAlgParameters()); + keySpec = keyInfo.getKeySpec(cipher); + } else { + byte[] pkcs8EncodedBytes = Base64.getDecoder().decode(privateKeyContents); + keySpec = new PKCS8EncodedKeySpec(pkcs8EncodedBytes); + } + + KeyFactory keyFactory = KeyFactory.getInstance("RSA"); + return keyFactory.generatePrivate(keySpec); + } + + public static Signature getSignature(String algorithm) throws GeneralSecurityException { + if (algorithm.equalsIgnoreCase(TOKEN_SIGNING_ALGORITHM_RS256)) { + return Signature.getInstance("SHA256withRSA"); + } else if (algorithm.equalsIgnoreCase(TOKEN_SIGNING_ALGORITHM_ES256)) { + return Signature.getInstance("SHA256withECDSA"); + } else { + throw new NoSuchAlgorithmException(String.format("Unsupported signing algorithm: %s", algorithm)); + } + } + + public static String sign(String algorithm, PrivateKey privateKey, String contentToSign) throws GeneralSecurityException { + Signature signature = getSignature(algorithm); + signature.initSign(privateKey); + signature.update(contentToSign.getBytes(StandardCharsets.UTF_8)); + byte[] signedContent = signature.sign(); + return Base64.getUrlEncoder().withoutPadding().encodeToString(signedContent); + } + + public static Optional staticAssertionJwtTemplate(ConfigurationUtils cu) { + if (cu.containsKey(SASL_OAUTHBEARER_ASSERTION_CLAIM_AUD) || + cu.containsKey(SASL_OAUTHBEARER_ASSERTION_CLAIM_ISS) || + cu.containsKey(SASL_OAUTHBEARER_ASSERTION_CLAIM_SUB)) { + Map staticClaimsPayload = new HashMap<>(); + + if (cu.containsKey(SASL_OAUTHBEARER_ASSERTION_CLAIM_AUD)) + staticClaimsPayload.put("aud", cu.validateString(SASL_OAUTHBEARER_ASSERTION_CLAIM_AUD)); + + if (cu.containsKey(SASL_OAUTHBEARER_ASSERTION_CLAIM_ISS)) + staticClaimsPayload.put("iss", cu.validateString(SASL_OAUTHBEARER_ASSERTION_CLAIM_ISS)); + + if (cu.containsKey(SASL_OAUTHBEARER_ASSERTION_CLAIM_SUB)) + staticClaimsPayload.put("sub", cu.validateString(SASL_OAUTHBEARER_ASSERTION_CLAIM_SUB)); + + Map header = Map.of(); + return Optional.of(new StaticAssertionJwtTemplate(header, staticClaimsPayload)); + } else { + return Optional.empty(); + } + } + + public static Optional fileAssertionJwtTemplate(ConfigurationUtils cu) { + if (cu.containsKey(SASL_OAUTHBEARER_ASSERTION_TEMPLATE_FILE)) { + File assertionTemplateFile = cu.validateFile(SASL_OAUTHBEARER_ASSERTION_TEMPLATE_FILE); + return Optional.of(new FileAssertionJwtTemplate(assertionTemplateFile)); + } else { + return Optional.empty(); + } + } + + public static DynamicAssertionJwtTemplate dynamicAssertionJwtTemplate(ConfigurationUtils cu, Time time) { + String algorithm = cu.validateString(SASL_OAUTHBEARER_ASSERTION_ALGORITHM); + int expSeconds = cu.validateInteger(SASL_OAUTHBEARER_ASSERTION_CLAIM_EXP_SECONDS, true); + int nbfSeconds = cu.validateInteger(SASL_OAUTHBEARER_ASSERTION_CLAIM_NBF_SECONDS, true); + boolean includeJti = cu.validateBoolean(SASL_OAUTHBEARER_ASSERTION_CLAIM_JTI_INCLUDE, true); + return new DynamicAssertionJwtTemplate(time, algorithm, expSeconds, nbfSeconds, includeJti); + } + + public static LayeredAssertionJwtTemplate layeredAssertionJwtTemplate(ConfigurationUtils cu, Time time) { + List templates = new ArrayList<>(); + staticAssertionJwtTemplate(cu).ifPresent(templates::add); + fileAssertionJwtTemplate(cu).ifPresent(templates::add); + templates.add(dynamicAssertionJwtTemplate(cu, time)); + return new LayeredAssertionJwtTemplate(templates); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/DefaultAssertionCreator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/DefaultAssertionCreator.java new file mode 100644 index 00000000000..db562fade87 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/DefaultAssertionCreator.java @@ -0,0 +1,96 @@ +/* + * 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.kafka.common.security.oauthbearer.internals.secured.assertion; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.security.oauthbearer.internals.secured.CachedFile; +import org.apache.kafka.common.utils.Utils; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.security.GeneralSecurityException; +import java.security.PrivateKey; +import java.util.Base64; +import java.util.Optional; + +import static org.apache.kafka.common.security.oauthbearer.internals.secured.CachedFile.RefreshPolicy.lastModifiedPolicy; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.assertion.AssertionUtils.privateKey; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.assertion.AssertionUtils.sign; + +/** + * This is the "default" {@link AssertionCreator} in that it is the common case of using a configured signing + * algorithm, private key file, and optional passphrase to sign a JWT to dynamically create an assertion. + * + *

    + * + * The provided private key file will be cached in memory but will be refreshed when the file changes. + * Note: there is not yet a facility to reload the configured passphrase. If using a private key + * passphrase, either use the same passphrase for each private key or else restart the client/application + * so that the new private key and passphrase will be used. + */ +public class DefaultAssertionCreator implements AssertionCreator { + + private static final Base64.Encoder BASE64_ENCODER = Base64.getUrlEncoder().withoutPadding(); + private final String algorithm; + private final CachedFile privateKeyFile; + + public DefaultAssertionCreator(String algorithm, File privateKeyFile, Optional passphrase) { + this.algorithm = algorithm; + + this.privateKeyFile = new CachedFile<>( + privateKeyFile, + new PrivateKeyTransformer(passphrase), + lastModifiedPolicy() + ); + } + + @Override + public String create(AssertionJwtTemplate template) throws GeneralSecurityException, IOException { + ObjectMapper mapper = new ObjectMapper(); + String header = BASE64_ENCODER.encodeToString(Utils.utf8(mapper.writeValueAsString(template.header()))); + String payload = BASE64_ENCODER.encodeToString(Utils.utf8(mapper.writeValueAsString(template.payload()))); + String content = header + "." + payload; + PrivateKey privateKey = privateKeyFile.transformed(); + String signedContent = sign(algorithm, privateKey, content); + return content + "." + signedContent; + } + + private static class PrivateKeyTransformer implements CachedFile.Transformer { + + private final Optional passphrase; + + public PrivateKeyTransformer(Optional passphrase) { + this.passphrase = passphrase; + } + + @Override + public PrivateKey transform(File file, String contents) { + try { + contents = contents.replace("-----BEGIN PRIVATE KEY-----", "") + .replace("-----END PRIVATE KEY-----", "") + .replace("\n", ""); + + return privateKey(contents.getBytes(StandardCharsets.UTF_8), passphrase); + } catch (GeneralSecurityException | IOException e) { + throw new KafkaException("An error occurred generating the OAuth assertion private key from " + file.getPath(), e); + } + } + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/DynamicAssertionJwtTemplate.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/DynamicAssertionJwtTemplate.java new file mode 100644 index 00000000000..ef1f45e4d1c --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/DynamicAssertionJwtTemplate.java @@ -0,0 +1,81 @@ +/* + * 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.kafka.common.security.oauthbearer.internals.secured.assertion; + +import org.apache.kafka.common.utils.Time; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; + +/** + * A "dynamic" {@link AssertionJwtTemplate} is that which will dynamically add the following values + * at runtime: + * + *

      + *
    • {@code alg} (Algorithm) header
    • + *
    • {@code typ} (Type) header
    • + *
    • {@code iat} (Issued at) timestamp claim (in seconds)
    • + *
    • {@code exp} (Expiration) timestamp claim (in seconds)
    • + *
    • {@code nbf} (Not before) timestamp claim (in seconds)
    • + *
    • (Optionally) {@code jti} (JWT ID) claim
    • + *
    + */ +public class DynamicAssertionJwtTemplate implements AssertionJwtTemplate { + + private final Time time; + private final String algorithm; + private final int expSeconds; + private final int nbfSeconds; + private final boolean includeJti; + + public DynamicAssertionJwtTemplate(Time time, + String algorithm, + int expSeconds, + int nbfSeconds, + boolean includeJti) { + this.time = time; + this.algorithm = algorithm; + this.expSeconds = expSeconds; + this.nbfSeconds = nbfSeconds; + this.includeJti = includeJti; + } + + @Override + public Map header() { + Map values = new HashMap<>(); + values.put("alg", algorithm); + values.put("typ", "JWT"); + return Collections.unmodifiableMap(values); + } + + @Override + public Map payload() { + long currentTimeSecs = time.milliseconds() / 1000L; + + Map values = new HashMap<>(); + values.put("iat", currentTimeSecs); + values.put("exp", currentTimeSecs + expSeconds); + values.put("nbf", currentTimeSecs - nbfSeconds); + + if (includeJti) + values.put("jti", UUID.randomUUID().toString()); + + return Collections.unmodifiableMap(values); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/FileAssertionCreator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/FileAssertionCreator.java new file mode 100644 index 00000000000..a6eb1eb2208 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/FileAssertionCreator.java @@ -0,0 +1,44 @@ +/* + * 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.kafka.common.security.oauthbearer.internals.secured.assertion; + +import org.apache.kafka.common.security.oauthbearer.internals.secured.CachedFile; + +import java.io.File; +import java.io.IOException; +import java.security.GeneralSecurityException; + +import static org.apache.kafka.common.security.oauthbearer.internals.secured.CachedFile.RefreshPolicy.lastModifiedPolicy; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.CachedFile.STRING_JSON_VALIDATING_TRANSFORMER; + +/** + * An {@link AssertionCreator} which takes a file from which the pre-created assertion is loaded and returned. + * If the file changes on disk, it will be reloaded in memory without needing to restart the client/application. + */ +public class FileAssertionCreator implements AssertionCreator { + + private final CachedFile assertionFile; + + public FileAssertionCreator(File assertionFile) { + this.assertionFile = new CachedFile<>(assertionFile, STRING_JSON_VALIDATING_TRANSFORMER, lastModifiedPolicy()); + } + + @Override + public String create(AssertionJwtTemplate ignored) throws GeneralSecurityException, IOException { + return assertionFile.transformed(); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/FileAssertionJwtTemplate.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/FileAssertionJwtTemplate.java new file mode 100644 index 00000000000..83c82feb015 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/FileAssertionJwtTemplate.java @@ -0,0 +1,165 @@ +/* + * 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.kafka.common.security.oauthbearer.internals.secured.assertion; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.security.oauthbearer.internals.secured.CachedFile; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import java.io.File; +import java.util.Collections; +import java.util.Map; + +import static org.apache.kafka.common.security.oauthbearer.internals.secured.CachedFile.RefreshPolicy.lastModifiedPolicy; + +/** + * {@code FileAssertionJwtTemplate} is used by the user to specify a JSON file on disk that contains static values + * that can be loaded and used to construct the assertion. The file structure is a JSON containing optionally a + * header and/or payload top-level attribute. + * + *

    + * + * Here is a minimally viable JSON structure: + * + *

    + * {
    + * }
    + * 
    + * + * OK, at that point it doesn't make sense for the user to build that file. + * + *

    + * + * Here is another, slightly less minimal JSON template: + * + *

    + * {
    + *    "header": {
    + *     "foo": 1
    + *   },
    + *    "payload": {
    + *     "bar": 2
    + *   }
    + * }
    + * 
    + * + * This provides a single header value and a single payload claim. + * + *

    + * + * A more realistic example template looks like so: + * + *

    + * {
    + *   "header": {
    + *     "kid": "f829d41b06f14f9e",
    + *     "some-random-header": 123456
    + *   },
    + *   "payload": {
    + *     "sub": "some-service-account",
    + *     "aud": "my_audience",
    + *     "iss": "https://example.com",
    + *     "useSomeResource": false,
    + *     "allowedAnimals": [
    + *       "cat",
    + *       "dog",
    + *       "hamster"
    + *     ]
    + *   }
    + * }
    + * 
    + * + * The AssertionCreator would accept the AssertionJwtTemplate and augment the template header and/or payload + * with dynamic values. For example, the above header would be augmented with the {@code alg} (algorithm) and + * {@code typ} (type) values per the OAuth RFC: + * + *
    + * {
    + *   "kid": "f829d41b06f14f9e",
    + *   "some-random-header": 123456,
    + *   "alg": "RS256",
    + *   "typ": "JWT"
    + * }
    + * 
    + * + * And the payload would also be augmented to add the {@code iat} (issued at) and {@code exp} (expiration) timestamps: + * + *
    + * {
    + *   "iat": 1741121401,
    + *   "exp": 1741125001,
    + *   "sub": "some-service-account",
    + *   "aud": "my_audience",
    + *   "iss": "https://example.com",
    + *   "useSomeResource": false,
    + *   "allowedAnimals": [
    + *     "cat",
    + *     "dog",
    + *     "hamster"
    + *   ]
    + * }
    + * 
    + */ +public class FileAssertionJwtTemplate implements AssertionJwtTemplate { + + @SuppressWarnings("unchecked") + private static final CachedFile.Transformer JSON_TRANSFORMER = (file, json) -> { + try { + ObjectMapper mapper = new ObjectMapper(); + Map map = (Map) mapper.readValue(json, Map.class); + + Map header = (Map) map.computeIfAbsent("header", k -> Map.of()); + Map payload = (Map) map.computeIfAbsent("payload", k -> Map.of()); + + return new CachedJwtTemplate(header, payload); + } catch (Exception e) { + throw new KafkaException("An error occurred parsing the OAuth assertion template file from " + file.getPath(), e); + } + }; + + private final CachedFile jsonFile; + + public FileAssertionJwtTemplate(File jsonFile) { + this.jsonFile = new CachedFile<>(jsonFile, JSON_TRANSFORMER, lastModifiedPolicy()); + } + + @Override + public Map header() { + return jsonFile.transformed().header; + } + + @Override + public Map payload() { + return jsonFile.transformed().payload; + } + + /** + * Internally, the cached file is represented by the two maps for the header and payload. + */ + private static class CachedJwtTemplate { + + private final Map header; + + private final Map payload; + + private CachedJwtTemplate(Map header, Map payload) { + this.header = Collections.unmodifiableMap(header); + this.payload = Collections.unmodifiableMap(payload); + } + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/LayeredAssertionJwtTemplate.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/LayeredAssertionJwtTemplate.java new file mode 100644 index 00000000000..847b622f97d --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/LayeredAssertionJwtTemplate.java @@ -0,0 +1,108 @@ +/* + * 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.kafka.common.security.oauthbearer.internals.secured.assertion; + +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.utils.Utils; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * This {@link AssertionJwtTemplate} layers multiple templates to produce an aggregated template. + * This is used, in practice, to achieve a layered approach where templates added later take precedence + * over templates that appear earlier in the list. Take for example the following list of templates, + * added in this order: + * + *
      + *
    1. Static/configuration-based JWT headers and claims via {@link StaticAssertionJwtTemplate}
    2. + *
    3. File-based JWT headers and claims via {@link FileAssertionJwtTemplate}
    4. + *
    5. Dynamic JWT headers and claims via {@link DynamicAssertionJwtTemplate}
    6. + *
    + * + * The templates are specified in ascending order of precedence. That is, in the list, a template with + * a list index of N+1 will effectively overwrite values provided by template at index N. + * In the above example, the {@link DynamicAssertionJwtTemplate} (index 2) will overwrite any values + * specified by the {@link FileAssertionJwtTemplate} (index 1), which will in turn overwrite any values + * from the {@link StaticAssertionJwtTemplate}. + * + *

    + * + * In practice, there shouldn't be much in the way of overwriting. The headers and claims provided + * by each layer are mostly distinct. For example, a {@link StaticAssertionJwtTemplate} loads values + * mainly from the configuration, such as the iss (Issuer) claim + * ({@link SaslConfigs#SASL_OAUTHBEARER_ASSERTION_CLAIM_ISS}). The iss claim probably + * doesn't change all that often, statically configuring it is sensible. However, other values, such + * as the exp (Expires) claim changes dynamically over time. Specifying a static expiration + * value doesn't make much sense. + * + *

    + * + * There are probably cases where it may make sense to overwrite static configuration with values that + * are more up-to-date. In that case, the {@link FileAssertionJwtTemplate} allows the user to provide + * headers and claims via a file that can be reloaded when it is modified. So, for example, if the value + * of the iss (Issuer) claim changes temporarily, the user can update the assertion + * template file ({@link SaslConfigs#SASL_OAUTHBEARER_ASSERTION_TEMPLATE_FILE}) to add an + * iss claim. In so doing, the template file will be reloaded, the + * {@code FileAssertionJwtTemplate} will overwrite the claim value in the generated assertion, and the + * client/application does not need to be restarted for the new value to take effect. Likewise, when the + * iss claim needs to be changed back to its normal value, the user can either update the + * template file with the new value, or simply remove the claim from the file altogether so that the + * original, static claim value is restored. + */ +public class LayeredAssertionJwtTemplate implements AssertionJwtTemplate { + + private final List templates; + + public LayeredAssertionJwtTemplate(AssertionJwtTemplate... templates) { + this.templates = Arrays.asList(templates); + } + + public LayeredAssertionJwtTemplate(List templates) { + this.templates = Collections.unmodifiableList(templates); + } + + @Override + public Map header() { + Map header = new HashMap<>(); + + for (AssertionJwtTemplate template : templates) + header.putAll(template.header()); + + return Collections.unmodifiableMap(header); + } + + @Override + public Map payload() { + Map payload = new HashMap<>(); + + for (AssertionJwtTemplate template : templates) + payload.putAll(template.payload()); + + return Collections.unmodifiableMap(payload); + } + + @Override + public void close() { + for (AssertionJwtTemplate template : templates) { + Utils.closeQuietly(template, "JWT assertion template"); + } + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/StaticAssertionJwtTemplate.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/StaticAssertionJwtTemplate.java new file mode 100644 index 00000000000..6d668f64064 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/StaticAssertionJwtTemplate.java @@ -0,0 +1,52 @@ +/* + * 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.kafka.common.security.oauthbearer.internals.secured.assertion; + +import java.util.Collections; +import java.util.Map; + +/** + * This {@link AssertionJwtTemplate} uses a static set of headers and claims provided on initialization. + * The values typically come from configuration, and it is often used in conjunction with other templates + * such as {@link LayeredAssertionJwtTemplate}. + */ +public class StaticAssertionJwtTemplate implements AssertionJwtTemplate { + + private final Map header; + + private final Map payload; + + public StaticAssertionJwtTemplate() { + this.header = Map.of(); + this.payload = Map.of(); + } + + public StaticAssertionJwtTemplate(Map header, Map payload) { + this.header = Collections.unmodifiableMap(header); + this.payload = Collections.unmodifiableMap(payload); + } + + @Override + public Map header() { + return header; + } + + @Override + public Map payload() { + return payload; + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerJwtValidatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/BrokerJwtValidatorTest.java similarity index 79% rename from clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerJwtValidatorTest.java rename to clients/src/test/java/org/apache/kafka/common/security/oauthbearer/BrokerJwtValidatorTest.java index 3b06bf07dec..5f76f508513 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerJwtValidatorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/BrokerJwtValidatorTest.java @@ -15,29 +15,28 @@ * limitations under the License. */ -package org.apache.kafka.common.security.oauthbearer.internals.secured; +package org.apache.kafka.common.security.oauthbearer; -import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken; +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenBuilder; +import org.apache.kafka.common.security.oauthbearer.internals.secured.CloseableVerificationKeyResolver; import org.jose4j.jwk.PublicJsonWebKey; import org.jose4j.jws.AlgorithmIdentifiers; import org.jose4j.lang.InvalidAlgorithmException; import org.junit.jupiter.api.Test; -import java.util.Collections; +import java.util.Map; +import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM; import static org.junit.jupiter.api.Assertions.assertEquals; public class BrokerJwtValidatorTest extends JwtValidatorTest { @Override protected JwtValidator createJwtValidator(AccessTokenBuilder builder) { - return new BrokerJwtValidator(30, - Collections.emptySet(), - null, - (jws, nestingContext) -> builder.jwk().getKey(), - builder.scopeClaimName(), - builder.subjectClaimName()); + CloseableVerificationKeyResolver resolver = (jws, nestingContext) -> builder.jwk().getKey(); + return new BrokerJwtValidator(resolver); } @Test @@ -73,6 +72,8 @@ public class BrokerJwtValidatorTest extends JwtValidatorTest { .subjectClaimName(subClaimName) .subject(null); JwtValidator validator = createJwtValidator(tokenBuilder); + Map saslConfigs = getSaslConfigs(SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME, subClaimName); + validator.configure(saslConfigs, OAUTHBEARER_MECHANISM, getJaasConfigEntries()); // Validation should succeed (e.g. signature verification) even if sub claim is missing OAuthBearerToken token = validator.validate(tokenBuilder.build()); @@ -83,6 +84,7 @@ public class BrokerJwtValidatorTest extends JwtValidatorTest { private void testEncryptionAlgorithm(PublicJsonWebKey jwk, String alg) throws Exception { AccessTokenBuilder builder = new AccessTokenBuilder().jwk(jwk).alg(alg); JwtValidator validator = createJwtValidator(builder); + validator.configure(getSaslConfigs(), OAUTHBEARER_MECHANISM, getJaasConfigEntries()); String accessToken = builder.build(); OAuthBearerToken token = validator.validate(accessToken); diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClaimValidationUtilsTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/ClaimValidationUtilsTest.java similarity index 72% rename from clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClaimValidationUtilsTest.java rename to clients/src/test/java/org/apache/kafka/common/security/oauthbearer/ClaimValidationUtilsTest.java index 89387797cdc..e468b93ba61 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClaimValidationUtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/ClaimValidationUtilsTest.java @@ -15,7 +15,10 @@ * limitations under the License. */ -package org.apache.kafka.common.security.oauthbearer.internals.secured; +package org.apache.kafka.common.security.oauthbearer; + +import org.apache.kafka.common.security.oauthbearer.internals.secured.ClaimValidationUtils; +import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerTest; import org.junit.jupiter.api.Test; @@ -42,15 +45,15 @@ public class ClaimValidationUtilsTest extends OAuthBearerTest { @Test public void testValidateScopesDisallowsDuplicates() { - assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateScopes("scope", Arrays.asList("a", "b", "a"))); - assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateScopes("scope", Arrays.asList("a", "b", " a "))); + assertThrows(JwtValidatorException.class, () -> ClaimValidationUtils.validateScopes("scope", Arrays.asList("a", "b", "a"))); + assertThrows(JwtValidatorException.class, () -> ClaimValidationUtils.validateScopes("scope", Arrays.asList("a", "b", " a "))); } @Test public void testValidateScopesDisallowsEmptyNullAndWhitespace() { - assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateScopes("scope", Arrays.asList("a", ""))); - assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateScopes("scope", Arrays.asList("a", null))); - assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateScopes("scope", Arrays.asList("a", " "))); + assertThrows(JwtValidatorException.class, () -> ClaimValidationUtils.validateScopes("scope", Arrays.asList("a", ""))); + assertThrows(JwtValidatorException.class, () -> ClaimValidationUtils.validateScopes("scope", Arrays.asList("a", null))); + assertThrows(JwtValidatorException.class, () -> ClaimValidationUtils.validateScopes("scope", Arrays.asList("a", " "))); } @Test @@ -100,12 +103,12 @@ public class ClaimValidationUtilsTest extends OAuthBearerTest { @Test public void testValidateExpirationDisallowsNull() { - assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateExpiration("exp", null)); + assertThrows(JwtValidatorException.class, () -> ClaimValidationUtils.validateExpiration("exp", null)); } @Test public void testValidateExpirationDisallowsNegatives() { - assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateExpiration("exp", -1L)); + assertThrows(JwtValidatorException.class, () -> ClaimValidationUtils.validateExpiration("exp", -1L)); } @Test @@ -117,9 +120,9 @@ public class ClaimValidationUtilsTest extends OAuthBearerTest { @Test public void testValidateSubjectDisallowsEmptyNullAndWhitespace() { - assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateSubject("sub", "")); - assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateSubject("sub", null)); - assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateSubject("sub", " ")); + assertThrows(JwtValidatorException.class, () -> ClaimValidationUtils.validateSubject("sub", "")); + assertThrows(JwtValidatorException.class, () -> ClaimValidationUtils.validateSubject("sub", null)); + assertThrows(JwtValidatorException.class, () -> ClaimValidationUtils.validateSubject("sub", " ")); } @Test @@ -131,9 +134,9 @@ public class ClaimValidationUtilsTest extends OAuthBearerTest { @Test public void testValidateClaimNameOverrideDisallowsEmptyNullAndWhitespace() { - assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateSubject("sub", "")); - assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateSubject("sub", null)); - assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateSubject("sub", " ")); + assertThrows(JwtValidatorException.class, () -> ClaimValidationUtils.validateSubject("sub", "")); + assertThrows(JwtValidatorException.class, () -> ClaimValidationUtils.validateSubject("sub", null)); + assertThrows(JwtValidatorException.class, () -> ClaimValidationUtils.validateSubject("sub", " ")); } @Test @@ -159,7 +162,7 @@ public class ClaimValidationUtilsTest extends OAuthBearerTest { @Test public void testValidateIssuedAtDisallowsNegatives() { - assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateIssuedAt("iat", -1L)); + assertThrows(JwtValidatorException.class, () -> ClaimValidationUtils.validateIssuedAt("iat", -1L)); } } diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientJwtValidatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/ClientJwtValidatorTest.java similarity index 83% rename from clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientJwtValidatorTest.java rename to clients/src/test/java/org/apache/kafka/common/security/oauthbearer/ClientJwtValidatorTest.java index 280aecd82c3..3156d3ca810 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientJwtValidatorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/ClientJwtValidatorTest.java @@ -15,13 +15,15 @@ * limitations under the License. */ -package org.apache.kafka.common.security.oauthbearer.internals.secured; +package org.apache.kafka.common.security.oauthbearer; + +import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenBuilder; public class ClientJwtValidatorTest extends JwtValidatorTest { @Override protected JwtValidator createJwtValidator(AccessTokenBuilder builder) { - return new ClientJwtValidator(builder.scopeClaimName(), builder.subjectClaimName()); + return new ClientJwtValidator(); } } diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetrieverTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtRetrieverTest.java similarity index 60% rename from clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetrieverTest.java rename to clients/src/test/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtRetrieverTest.java index 83fd57713b0..72d52b4fb5c 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetrieverTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtRetrieverTest.java @@ -15,10 +15,11 @@ * limitations under the License. */ -package org.apache.kafka.common.security.oauthbearer.internals.secured; +package org.apache.kafka.common.security.oauthbearer; import org.apache.kafka.common.config.ConfigException; -import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule; +import org.apache.kafka.common.security.oauthbearer.internals.secured.ConfigurationUtils; +import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerTest; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; @@ -39,6 +40,8 @@ import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG; import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_ID_CONFIG; import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_SECRET_CONFIG; +import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM; +import static org.apache.kafka.test.TestUtils.tempFile; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertInstanceOf; @@ -52,17 +55,13 @@ public class DefaultJwtRetrieverTest extends OAuthBearerTest { @Test public void testConfigureRefreshingFileJwtRetriever() throws Exception { - String expected = "{}"; + String expected = createJwt("jdoe"); + String file = tempFile(expected).toURI().toString(); + System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, file); + Map configs = Collections.singletonMap(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, file); - File tmpDir = createTempDir("access-token"); - File accessTokenFile = createTempFile(tmpDir, "access-token-", ".json", expected); - - System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, accessTokenFile.toURI().toString()); - Map configs = Collections.singletonMap(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, accessTokenFile.toURI().toString()); - Map jaasConfig = Collections.emptyMap(); - - try (JwtRetriever jwtRetriever = new DefaultJwtRetriever(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, jaasConfig)) { - jwtRetriever.init(); + try (JwtRetriever jwtRetriever = new DefaultJwtRetriever()) { + jwtRetriever.configure(configs, OAUTHBEARER_MECHANISM, getJaasConfigEntries()); assertEquals(expected, jwtRetriever.retrieve()); } } @@ -73,80 +72,63 @@ public class DefaultJwtRetrieverTest extends OAuthBearerTest { String file = new File("/tmp/this-directory-does-not-exist/foo.json").toURI().toString(); System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, file); Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, file); - Map jaasConfig = Collections.emptyMap(); - try (JwtRetriever jwtRetriever = new DefaultJwtRetriever(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, jaasConfig)) { - assertThrowsWithMessage(ConfigException.class, jwtRetriever::init, "that doesn't exist"); - } - } - - @Test - public void testConfigureRefreshingFileJwtRetrieverWithInvalidFile() throws Exception { - // Should fail because while the parent path exists, the file itself doesn't. - File tmpDir = createTempDir("this-directory-does-exist"); - File accessTokenFile = new File(tmpDir, "this-file-does-not-exist.json"); - System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, accessTokenFile.toURI().toString()); - Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, accessTokenFile.toURI().toString()); - Map jaasConfig = Collections.emptyMap(); - - try (JwtRetriever jwtRetriever = new DefaultJwtRetriever(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, jaasConfig)) { - assertThrowsWithMessage(ConfigException.class, jwtRetriever::init, "that doesn't exist"); + try (JwtRetriever jwtRetriever = new DefaultJwtRetriever()) { + assertThrowsWithMessage( + ConfigException.class, + () -> jwtRetriever.configure(configs, OAUTHBEARER_MECHANISM, getJaasConfigEntries()), + "that doesn't exist" + ); } } @Test public void testSaslOauthbearerTokenEndpointUrlIsNotAllowed() throws Exception { - // Should fail if the URL is not allowed - File tmpDir = createTempDir("not_allowed"); - File accessTokenFile = new File(tmpDir, "not_allowed.json"); - Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, accessTokenFile.toURI().toString()); + // Should fail because the URL was not allowed + String file = tempFile("test data").toURI().toString(); + Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, file); - try (JwtRetriever jwtRetriever = new DefaultJwtRetriever(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, Collections.emptyMap())) { - assertThrowsWithMessage(ConfigException.class, jwtRetriever::init, ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG); + try (JwtRetriever jwtRetriever = new DefaultJwtRetriever()) { + assertThrowsWithMessage( + ConfigException.class, + () -> jwtRetriever.configure(configs, OAUTHBEARER_MECHANISM, getJaasConfigEntries()), + ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG + ); } } @Test public void testConfigureWithAccessTokenFile() throws Exception { - String expected = "{}"; + String expected = createJwt("jdoe"); + String file = tempFile(expected).toURI().toString(); + System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, file); + Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, file); - File tmpDir = createTempDir("access-token"); - File accessTokenFile = createTempFile(tmpDir, "access-token-", ".json", expected); - System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, accessTokenFile.toURI().toString()); - - Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, accessTokenFile.toURI().toString()); - - DefaultJwtRetriever jwtRetriever = new DefaultJwtRetriever( - configs, - OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, - Map.of() - ); - assertDoesNotThrow(jwtRetriever::init); - assertInstanceOf(FileJwtRetriever.class, jwtRetriever.delegate()); + try (DefaultJwtRetriever jwtRetriever = new DefaultJwtRetriever()) { + assertDoesNotThrow(() -> jwtRetriever.configure(configs, OAUTHBEARER_MECHANISM, getJaasConfigEntries())); + assertInstanceOf(FileJwtRetriever.class, jwtRetriever.delegate()); + } } @Test - public void testConfigureWithAccessClientCredentials() { + public void testConfigureWithAccessClientCredentials() throws Exception { Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, "http://www.example.com"); System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, "http://www.example.com"); Map jaasConfigs = new HashMap<>(); jaasConfigs.put(CLIENT_ID_CONFIG, "an ID"); jaasConfigs.put(CLIENT_SECRET_CONFIG, "a secret"); - DefaultJwtRetriever jwtRetriever = new DefaultJwtRetriever( - configs, - OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, - jaasConfigs - ); - assertDoesNotThrow(jwtRetriever::init); - assertInstanceOf(HttpJwtRetriever.class, jwtRetriever.delegate()); + try (DefaultJwtRetriever jwtRetriever = new DefaultJwtRetriever()) { + assertDoesNotThrow(() -> jwtRetriever.configure(configs, OAUTHBEARER_MECHANISM, getJaasConfigEntries(jaasConfigs))); + assertInstanceOf(ClientCredentialsJwtRetriever.class, jwtRetriever.delegate()); + } } @ParameterizedTest @MethodSource("urlencodeHeaderSupplier") public void testUrlencodeHeader(Map configs, boolean expectedValue) { ConfigurationUtils cu = new ConfigurationUtils(configs); - boolean actualValue = DefaultJwtRetriever.validateUrlencodeHeader(cu); + boolean actualValue = ClientCredentialsJwtRetriever.validateUrlencodeHeader(cu); assertEquals(expectedValue, actualValue); } @@ -158,5 +140,4 @@ public class DefaultJwtRetrieverTest extends OAuthBearerTest { Arguments.of(Collections.singletonMap(SASL_OAUTHBEARER_HEADER_URLENCODE, false), false) ); } - } diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtValidatorTest.java similarity index 74% rename from clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidatorTest.java rename to clients/src/test/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtValidatorTest.java index 9d136b72b14..14c33a012c8 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidatorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtValidatorTest.java @@ -15,15 +15,18 @@ * limitations under the License. */ -package org.apache.kafka.common.security.oauthbearer.internals.secured; +package org.apache.kafka.common.security.oauthbearer; -import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule; +import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenBuilder; +import org.apache.kafka.common.security.oauthbearer.internals.secured.CloseableVerificationKeyResolver; +import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerTest; import org.jose4j.jws.AlgorithmIdentifiers; import org.junit.jupiter.api.Test; import java.util.Map; +import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertInstanceOf; @@ -35,23 +38,16 @@ public class DefaultJwtValidatorTest extends OAuthBearerTest { .alg(AlgorithmIdentifiers.RSA_USING_SHA256); CloseableVerificationKeyResolver verificationKeyResolver = createVerificationKeyResolver(builder); Map configs = getSaslConfigs(); - DefaultJwtValidator jwtValidator = new DefaultJwtValidator( - configs, - OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, - verificationKeyResolver - ); - assertDoesNotThrow(jwtValidator::init); + DefaultJwtValidator jwtValidator = new DefaultJwtValidator(verificationKeyResolver); + assertDoesNotThrow(() -> jwtValidator.configure(configs, OAUTHBEARER_MECHANISM, getJaasConfigEntries())); assertInstanceOf(BrokerJwtValidator.class, jwtValidator.delegate()); } @Test public void testConfigureWithoutVerificationKeyResolver() { Map configs = getSaslConfigs(); - DefaultJwtValidator jwtValidator = new DefaultJwtValidator( - configs, - OAuthBearerLoginModule.OAUTHBEARER_MECHANISM - ); - assertDoesNotThrow(jwtValidator::init); + DefaultJwtValidator jwtValidator = new DefaultJwtValidator(); + assertDoesNotThrow(() -> jwtValidator.configure(configs, OAUTHBEARER_MECHANISM, getJaasConfigEntries())); assertInstanceOf(ClientJwtValidator.class, jwtValidator.delegate()); } diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/JwtBearerJwtRetrieverTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/JwtBearerJwtRetrieverTest.java new file mode 100644 index 00000000000..c466ac83689 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/JwtBearerJwtRetrieverTest.java @@ -0,0 +1,152 @@ +/* + * 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.kafka.common.security.oauthbearer; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerTest; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.security.GeneralSecurityException; +import java.util.List; +import java.util.Map; + +import javax.security.auth.login.AppConfigurationEntry; + +import static org.apache.kafka.common.config.SaslConfigs.DEFAULT_SASL_OAUTHBEARER_ASSERTION_ALGORITHM; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_ASSERTION_ALGORITHM; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_ASSERTION_FILE; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_FILE; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL; +import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_FILES_CONFIG; +import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG; +import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM; +import static org.apache.kafka.test.TestUtils.tempFile; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class JwtBearerJwtRetrieverTest extends OAuthBearerTest { + + @AfterEach + public void tearDown() throws Exception { + System.clearProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG); + System.clearProperty(ALLOWED_SASL_OAUTHBEARER_FILES_CONFIG); + } + + @Test + public void testConfigure() throws Exception { + String tokenEndpointUrl = "https://www.example.com"; + String privateKeyFile = generatePrivateKey().getPath(); + + System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, tokenEndpointUrl); + System.setProperty(ALLOWED_SASL_OAUTHBEARER_FILES_CONFIG, privateKeyFile); + + Map configs = getSaslConfigs( + Map.of( + SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, tokenEndpointUrl, + SASL_OAUTHBEARER_ASSERTION_ALGORITHM, DEFAULT_SASL_OAUTHBEARER_ASSERTION_ALGORITHM, + SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_FILE, privateKeyFile + ) + ); + + List jaasConfigEntries = getJaasConfigEntries(); + + try (JwtBearerJwtRetriever jwtRetriever = new JwtBearerJwtRetriever()) { + assertDoesNotThrow(() -> jwtRetriever.configure(configs, OAUTHBEARER_MECHANISM, jaasConfigEntries)); + } + } + + @Test + public void testConfigureWithMalformedPrivateKey() throws Exception { + String tokenEndpointUrl = "https://www.example.com"; + String malformedPrivateKeyFile = tempFile().getPath(); + + System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, tokenEndpointUrl); + System.setProperty(ALLOWED_SASL_OAUTHBEARER_FILES_CONFIG, malformedPrivateKeyFile); + + Map configs = getSaslConfigs( + Map.of( + SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, tokenEndpointUrl, + SASL_OAUTHBEARER_ASSERTION_ALGORITHM, DEFAULT_SASL_OAUTHBEARER_ASSERTION_ALGORITHM, + SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_FILE, malformedPrivateKeyFile + ) + ); + + List jaasConfigEntries = getJaasConfigEntries(); + + try (JwtBearerJwtRetriever jwtRetriever = new JwtBearerJwtRetriever()) { + KafkaException e = assertThrows(KafkaException.class, () -> jwtRetriever.configure(configs, OAUTHBEARER_MECHANISM, jaasConfigEntries)); + assertNotNull(e.getCause()); + assertInstanceOf(GeneralSecurityException.class, e.getCause()); + } + } + + @Test + public void testConfigureWithStaticAssertion() throws Exception { + String tokenEndpointUrl = "https://www.example.com"; + String assertionFile = tempFile(createJwt("jdoe")).getPath(); + + System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, tokenEndpointUrl); + System.setProperty(ALLOWED_SASL_OAUTHBEARER_FILES_CONFIG, assertionFile); + + Map configs = getSaslConfigs( + Map.of( + SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, tokenEndpointUrl, + SASL_OAUTHBEARER_ASSERTION_ALGORITHM, DEFAULT_SASL_OAUTHBEARER_ASSERTION_ALGORITHM, + SASL_OAUTHBEARER_ASSERTION_FILE, assertionFile + ) + ); + + List jaasConfigEntries = getJaasConfigEntries(); + + try (JwtBearerJwtRetriever jwtRetriever = new JwtBearerJwtRetriever()) { + assertDoesNotThrow(() -> jwtRetriever.configure(configs, OAUTHBEARER_MECHANISM, jaasConfigEntries)); + } + } + + @Test + public void testConfigureWithInvalidPassphrase() throws Exception { + String tokenEndpointUrl = "https://www.example.com"; + String privateKeyFile = generatePrivateKey().getPath(); + + System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, tokenEndpointUrl); + System.setProperty(ALLOWED_SASL_OAUTHBEARER_FILES_CONFIG, privateKeyFile); + + Map configs = getSaslConfigs( + Map.of( + SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, tokenEndpointUrl, + SASL_OAUTHBEARER_ASSERTION_ALGORITHM, DEFAULT_SASL_OAUTHBEARER_ASSERTION_ALGORITHM, + SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_FILE, privateKeyFile, + SaslConfigs.SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_PASSPHRASE, "this-passphrase-is-invalid" + ) + ); + + List jaasConfigEntries = getJaasConfigEntries(); + + try (JwtBearerJwtRetriever jwtRetriever = new JwtBearerJwtRetriever()) { + KafkaException e = assertThrows(KafkaException.class, () -> jwtRetriever.configure(configs, OAUTHBEARER_MECHANISM, jaasConfigEntries)); + assertNotNull(e.getCause()); + assertInstanceOf(IOException.class, e.getCause()); + } + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtValidatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/JwtValidatorTest.java similarity index 73% rename from clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtValidatorTest.java rename to clients/src/test/java/org/apache/kafka/common/security/oauthbearer/JwtValidatorTest.java index bfbf29d0266..09e01c42f3c 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtValidatorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/JwtValidatorTest.java @@ -15,7 +15,10 @@ * limitations under the License. */ -package org.apache.kafka.common.security.oauthbearer.internals.secured; +package org.apache.kafka.common.security.oauthbearer; + +import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenBuilder; +import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerTest; import org.jose4j.jws.AlgorithmIdentifiers; import org.jose4j.jwx.HeaderParameterNames; @@ -38,25 +41,25 @@ public abstract class JwtValidatorTest extends OAuthBearerTest { @Test public void testNull() throws Exception { JwtValidator validator = createJwtValidator(); - assertThrowsWithMessage(ValidateException.class, () -> validator.validate(null), "Malformed JWT provided; expected three sections (header, payload, and signature)"); + assertThrowsWithMessage(JwtValidatorException.class, () -> validator.validate(null), "Malformed JWT provided; expected three sections (header, payload, and signature)"); } @Test public void testEmptyString() throws Exception { JwtValidator validator = createJwtValidator(); - assertThrowsWithMessage(ValidateException.class, () -> validator.validate(""), "Malformed JWT provided; expected three sections (header, payload, and signature)"); + assertThrowsWithMessage(JwtValidatorException.class, () -> validator.validate(""), "Malformed JWT provided; expected three sections (header, payload, and signature)"); } @Test public void testWhitespace() throws Exception { JwtValidator validator = createJwtValidator(); - assertThrowsWithMessage(ValidateException.class, () -> validator.validate(" "), "Malformed JWT provided; expected three sections (header, payload, and signature)"); + assertThrowsWithMessage(JwtValidatorException.class, () -> validator.validate(" "), "Malformed JWT provided; expected three sections (header, payload, and signature)"); } @Test public void testEmptySections() throws Exception { JwtValidator validator = createJwtValidator(); - assertThrowsWithMessage(ValidateException.class, () -> validator.validate(".."), "Malformed JWT provided; expected three sections (header, payload, and signature)"); + assertThrowsWithMessage(JwtValidatorException.class, () -> validator.validate(".."), "Malformed JWT provided; expected three sections (header, payload, and signature)"); } @Test @@ -66,7 +69,7 @@ public abstract class JwtValidatorTest extends OAuthBearerTest { String payload = createBase64JsonJwtSection(node -> { }); String signature = ""; String accessToken = String.format("%s.%s.%s", header, payload, signature); - assertThrows(ValidateException.class, () -> validator.validate(accessToken)); + assertThrows(JwtValidatorException.class, () -> validator.validate(accessToken)); } @Test @@ -76,7 +79,7 @@ public abstract class JwtValidatorTest extends OAuthBearerTest { String payload = ""; String signature = ""; String accessToken = String.format("%s.%s.%s", header, payload, signature); - assertThrows(ValidateException.class, () -> validator.validate(accessToken)); + assertThrows(JwtValidatorException.class, () -> validator.validate(accessToken)); } @Test @@ -86,7 +89,7 @@ public abstract class JwtValidatorTest extends OAuthBearerTest { String payload = createBase64JsonJwtSection(node -> { }); String signature = ""; String accessToken = String.format("%s.%s.%s", header, payload, signature); - assertThrows(ValidateException.class, () -> validator.validate(accessToken)); + assertThrows(JwtValidatorException.class, () -> validator.validate(accessToken)); } } \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginCallbackHandlerTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginCallbackHandlerTest.java index 290c58d6553..54857cd8cc0 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginCallbackHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginCallbackHandlerTest.java @@ -21,23 +21,15 @@ import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.security.auth.SaslExtensionsCallback; import org.apache.kafka.common.security.oauthbearer.internals.OAuthBearerClientInitialResponse; import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenBuilder; -import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtRetriever; -import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtValidator; -import org.apache.kafka.common.security.oauthbearer.internals.secured.FileJwtRetriever; -import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtRetriever; -import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtValidator; import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerTest; import org.jose4j.jws.AlgorithmIdentifiers; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; -import java.io.File; import java.io.IOException; -import java.util.Calendar; import java.util.HashMap; import java.util.Map; -import java.util.TimeZone; import javax.security.auth.callback.Callback; import javax.security.auth.callback.UnsupportedCallbackException; @@ -46,6 +38,8 @@ import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG; import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_ID_CONFIG; import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_SECRET_CONFIG; +import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM; +import static org.apache.kafka.test.TestUtils.tempFile; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; @@ -68,9 +62,9 @@ public class OAuthBearerLoginCallbackHandlerTest extends OAuthBearerTest { .alg(AlgorithmIdentifiers.RSA_USING_SHA256); String accessToken = builder.build(); JwtRetriever jwtRetriever = () -> accessToken; - JwtValidator jwtValidator = createJwtValidator(configs); + JwtValidator jwtValidator = createJwtValidator(); OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - handler.init(Map.of(), jwtRetriever, jwtValidator); + handler.configure(configs, OAUTHBEARER_MECHANISM, getJaasConfigEntries(), jwtRetriever, jwtValidator); try { OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); @@ -98,10 +92,10 @@ public class OAuthBearerLoginCallbackHandlerTest extends OAuthBearerTest { jaasConfig.put("extension_bar", 2); jaasConfig.put("EXTENSION_baz", "3"); - JwtRetriever jwtRetriever = createJwtRetriever(configs, jaasConfig); - JwtValidator jwtValidator = createJwtValidator(configs); + JwtRetriever jwtRetriever = createJwtRetriever(); + JwtValidator jwtValidator = createJwtValidator(); OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - handler.init(jaasConfig, jwtRetriever, jwtValidator); + handler.configure(configs, OAUTHBEARER_MECHANISM, getJaasConfigEntries(jaasConfig), jwtRetriever, jwtValidator); try { SaslExtensionsCallback callback = new SaslExtensionsCallback(); @@ -129,10 +123,10 @@ public class OAuthBearerLoginCallbackHandlerTest extends OAuthBearerTest { jaasConfig.put(CLIENT_SECRET_CONFIG, "a secret"); jaasConfig.put(illegalKey, "this key isn't allowed per OAuthBearerClientInitialResponse.validateExtensions"); - JwtRetriever jwtRetriever = createJwtRetriever(configs, jaasConfig); - JwtValidator jwtValidator = createJwtValidator(configs); + JwtRetriever jwtRetriever = createJwtRetriever(); + JwtValidator jwtValidator = createJwtValidator(); OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - handler.init(jaasConfig, jwtRetriever, jwtValidator); + handler.configure(configs, OAUTHBEARER_MECHANISM, getJaasConfigEntries(jaasConfig), jwtRetriever, jwtValidator); try { SaslExtensionsCallback callback = new SaslExtensionsCallback(); @@ -148,9 +142,9 @@ public class OAuthBearerLoginCallbackHandlerTest extends OAuthBearerTest { public void testInvalidCallbackGeneratesUnsupportedCallbackException() { Map configs = getSaslConfigs(); JwtRetriever jwtRetriever = () -> "test"; - JwtValidator jwtValidator = createJwtValidator(configs); + JwtValidator jwtValidator = createJwtValidator(); OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - handler.init(Map.of(), jwtRetriever, jwtValidator); + handler.configure(configs, OAUTHBEARER_MECHANISM, getJaasConfigEntries(), jwtRetriever, jwtValidator); try { Callback unsupportedCallback = new Callback() { }; @@ -164,23 +158,23 @@ public class OAuthBearerLoginCallbackHandlerTest extends OAuthBearerTest { public void testInvalidAccessToken() throws Exception { testInvalidAccessToken("this isn't valid", "Malformed JWT provided"); testInvalidAccessToken("this.isn't.valid", "malformed Base64 URL encoded value"); - testInvalidAccessToken(createAccessKey("this", "isn't", "valid"), "malformed JSON"); - testInvalidAccessToken(createAccessKey("{}", "{}", "{}"), "exp value must be non-null"); + testInvalidAccessToken(createJwt("this", "isn't", "valid"), "malformed JSON"); + testInvalidAccessToken(createJwt("{}", "{}", "{}"), "exp value must be non-null"); } @Test public void testMissingAccessToken() { Map configs = getSaslConfigs(); JwtRetriever jwtRetriever = () -> { - throw new IOException("The token endpoint response access_token value must be non-null"); + throw new JwtRetrieverException("The token endpoint response access_token value must be non-null"); }; - JwtValidator jwtValidator = createJwtValidator(configs); + JwtValidator jwtValidator = createJwtValidator(); OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - handler.init(Map.of(), jwtRetriever, jwtValidator); + handler.configure(configs, OAUTHBEARER_MECHANISM, getJaasConfigEntries(), jwtRetriever, jwtValidator); try { OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); - assertThrowsWithMessage(IOException.class, + assertThrowsWithMessage(JwtRetrieverException.class, () -> handler.handle(new Callback[]{callback}), "token endpoint response access_token value must be non-null"); } finally { @@ -190,22 +184,17 @@ public class OAuthBearerLoginCallbackHandlerTest extends OAuthBearerTest { @Test public void testFileTokenRetrieverHandlesNewline() throws IOException { - Calendar cal = Calendar.getInstance(TimeZone.getTimeZone("UTC")); - long cur = cal.getTimeInMillis() / 1000; - String exp = "" + (cur + 60 * 60); // 1 hour in future - String iat = "" + cur; - - String expected = createAccessKey("{}", String.format("{\"exp\":%s, \"iat\":%s, \"sub\":\"subj\"}", exp, iat), "sign"); + String expected = createJwt("jdoe"); String withNewline = expected + "\n"; - File tmpDir = createTempDir("access-token"); - File accessTokenFile = createTempFile(tmpDir, "access-token-", ".json", withNewline); + String accessTokenFile = tempFile(withNewline).toURI().toString(); - Map configs = getSaslConfigs(); - JwtRetriever jwtRetriever = new FileJwtRetriever(accessTokenFile.toPath()); - JwtValidator jwtValidator = createJwtValidator(configs); + System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, accessTokenFile); + Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, accessTokenFile); + JwtRetriever jwtRetriever = new FileJwtRetriever(); + JwtValidator jwtValidator = createJwtValidator(); OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - handler.init(Map.of(), jwtRetriever, jwtValidator); + handler.configure(configs, OAUTHBEARER_MECHANISM, getJaasConfigEntries(), jwtRetriever, jwtValidator); OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); try { @@ -227,9 +216,9 @@ public class OAuthBearerLoginCallbackHandlerTest extends OAuthBearerTest { private void testInvalidAccessToken(String accessToken, String expectedMessageSubstring) throws Exception { Map configs = getSaslConfigs(); JwtRetriever jwtRetriever = () -> accessToken; - JwtValidator jwtValidator = createJwtValidator(configs); + JwtValidator jwtValidator = createJwtValidator(); OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - handler.init(Map.of(), jwtRetriever, jwtValidator); + handler.configure(configs, OAUTHBEARER_MECHANISM, getJaasConfigEntries(), jwtRetriever, jwtValidator); try { OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); @@ -246,15 +235,11 @@ public class OAuthBearerLoginCallbackHandlerTest extends OAuthBearerTest { } } - private static DefaultJwtRetriever createJwtRetriever(Map configs) { - return createJwtRetriever(configs, Map.of()); + private static DefaultJwtRetriever createJwtRetriever() { + return new DefaultJwtRetriever(); } - private static DefaultJwtRetriever createJwtRetriever(Map configs, Map jaasConfigs) { - return new DefaultJwtRetriever(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, jaasConfigs); - } - - private static DefaultJwtValidator createJwtValidator(Map configs) { - return new DefaultJwtValidator(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM); + private static DefaultJwtValidator createJwtValidator() { + return new DefaultJwtValidator(); } } diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandlerTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandlerTest.java index 0f1315b4281..adabec6bc95 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandlerTest.java @@ -20,10 +20,7 @@ package org.apache.kafka.common.security.oauthbearer; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenBuilder; import org.apache.kafka.common.security.oauthbearer.internals.secured.CloseableVerificationKeyResolver; -import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtValidator; -import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtValidator; import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerTest; -import org.apache.kafka.common.security.oauthbearer.internals.secured.ValidateException; import org.jose4j.jws.AlgorithmIdentifiers; import org.junit.jupiter.api.Test; @@ -34,8 +31,10 @@ import java.util.List; import java.util.Map; import javax.security.auth.callback.Callback; +import javax.security.auth.login.AppConfigurationEntry; import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE; +import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; @@ -57,9 +56,15 @@ public class OAuthBearerValidatorCallbackHandlerTest extends OAuthBearerTest { Map configs = getSaslConfigs(SASL_OAUTHBEARER_EXPECTED_AUDIENCE, allAudiences); CloseableVerificationKeyResolver verificationKeyResolver = createVerificationKeyResolver(builder); - JwtValidator jwtValidator = createJwtValidator(configs, verificationKeyResolver); + JwtValidator jwtValidator = createJwtValidator(verificationKeyResolver); OAuthBearerValidatorCallbackHandler handler = new OAuthBearerValidatorCallbackHandler(); - handler.init(verificationKeyResolver, jwtValidator); + handler.configure( + configs, + OAUTHBEARER_MECHANISM, + getJaasConfigEntries(), + verificationKeyResolver, + jwtValidator + ); try { OAuthBearerValidatorCallback callback = new OAuthBearerValidatorCallback(accessToken); @@ -83,25 +88,25 @@ public class OAuthBearerValidatorCallbackHandlerTest extends OAuthBearerTest { String substring = "invalid_token"; assertInvalidAccessTokenFails("this isn't valid", substring); assertInvalidAccessTokenFails("this.isn't.valid", substring); - assertInvalidAccessTokenFails(createAccessKey("this", "isn't", "valid"), substring); - assertInvalidAccessTokenFails(createAccessKey("{}", "{}", "{}"), substring); + assertInvalidAccessTokenFails(createJwt("this", "isn't", "valid"), substring); + assertInvalidAccessTokenFails(createJwt("{}", "{}", "{}"), substring); } @Test - public void testHandlerInitThrowsException() throws IOException { - IOException initError = new IOException("init() error"); + public void testHandlerConfigureThrowsException() throws IOException { + KafkaException configureError = new KafkaException("configure() error"); AccessTokenBuilder builder = new AccessTokenBuilder() .alg(AlgorithmIdentifiers.RSA_USING_SHA256); CloseableVerificationKeyResolver verificationKeyResolver = createVerificationKeyResolver(builder); JwtValidator jwtValidator = new JwtValidator() { @Override - public void init() throws IOException { - throw initError; + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + throw configureError; } @Override - public OAuthBearerToken validate(String accessToken) throws ValidateException { + public OAuthBearerToken validate(String accessToken) throws JwtValidatorException { return null; } }; @@ -109,12 +114,17 @@ public class OAuthBearerValidatorCallbackHandlerTest extends OAuthBearerTest { OAuthBearerValidatorCallbackHandler handler = new OAuthBearerValidatorCallbackHandler(); // An error initializing the JwtValidator should cause OAuthBearerValidatorCallbackHandler.init() to fail. - KafkaException root = assertThrows( + KafkaException error = assertThrows( KafkaException.class, - () -> handler.init(verificationKeyResolver, jwtValidator) + () -> handler.configure( + getSaslConfigs(), + OAUTHBEARER_MECHANISM, + getJaasConfigEntries(), + verificationKeyResolver, + jwtValidator + ) ); - assertNotNull(root.getCause()); - assertEquals(initError, root.getCause()); + assertEquals(configureError, error); } @Test @@ -129,13 +139,19 @@ public class OAuthBearerValidatorCallbackHandlerTest extends OAuthBearerTest { } @Override - public OAuthBearerToken validate(String accessToken) throws ValidateException { + public OAuthBearerToken validate(String accessToken) throws JwtValidatorException { return null; } }; OAuthBearerValidatorCallbackHandler handler = new OAuthBearerValidatorCallbackHandler(); - handler.init(verificationKeyResolver, jwtValidator); + handler.configure( + getSaslConfigs(), + OAUTHBEARER_MECHANISM, + getJaasConfigEntries(), + verificationKeyResolver, + jwtValidator + ); // An error closings the JwtValidator should *not* cause OAuthBearerValidatorCallbackHandler.close() to fail. assertDoesNotThrow(handler::close); @@ -146,9 +162,16 @@ public class OAuthBearerValidatorCallbackHandlerTest extends OAuthBearerTest { .alg(AlgorithmIdentifiers.RSA_USING_SHA256); Map configs = getSaslConfigs(); CloseableVerificationKeyResolver verificationKeyResolver = createVerificationKeyResolver(builder); - JwtValidator jwtValidator = createJwtValidator(configs, verificationKeyResolver); + JwtValidator jwtValidator = createJwtValidator(verificationKeyResolver); + OAuthBearerValidatorCallbackHandler handler = new OAuthBearerValidatorCallbackHandler(); - handler.init(verificationKeyResolver, jwtValidator); + handler.configure( + configs, + OAUTHBEARER_MECHANISM, + getJaasConfigEntries(), + verificationKeyResolver, + jwtValidator + ); try { OAuthBearerValidatorCallback callback = new OAuthBearerValidatorCallback(accessToken); @@ -163,8 +186,8 @@ public class OAuthBearerValidatorCallbackHandlerTest extends OAuthBearerTest { } } - private JwtValidator createJwtValidator(Map configs, CloseableVerificationKeyResolver verificationKeyResolver) { - return new DefaultJwtValidator(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, verificationKeyResolver); + private JwtValidator createJwtValidator(CloseableVerificationKeyResolver verificationKeyResolver) { + return new DefaultJwtValidator(verificationKeyResolver); } private CloseableVerificationKeyResolver createVerificationKeyResolver(AccessTokenBuilder builder) { diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenBuilder.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenBuilder.java index cc910e0d16c..b0828d5d281 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenBuilder.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenBuilder.java @@ -36,6 +36,10 @@ import java.util.Map; public class AccessTokenBuilder { + private final String scopeClaimName = "scope"; + + private final Long issuedAtSeconds; + private final ObjectMapper objectMapper = new ObjectMapper(); private String alg; @@ -48,10 +52,6 @@ public class AccessTokenBuilder { private Object scope = "engineering"; - private final String scopeClaimName = "scope"; - - private final Long issuedAtSeconds; - private Long expirationSeconds; private PublicJsonWebKey jwk; diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CachedFileTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CachedFileTest.java new file mode 100644 index 00000000000..e22056c663d --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CachedFileTest.java @@ -0,0 +1,151 @@ +/* + * 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.kafka.common.security.oauthbearer.internals.secured; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.utils.Utils; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.StandardOpenOption; +import java.util.List; + +import static org.apache.kafka.test.TestUtils.tempFile; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class CachedFileTest extends OAuthBearerTest { + + @Test + public void testStaticPolicy() throws Exception { + File tmpFile = tempFile(" foo "); + + CachedFile.Transformer transformer = (file, contents) -> contents.trim(); + CachedFile.RefreshPolicy refreshPolicy = CachedFile.RefreshPolicy.staticPolicy(); + CachedFile cachedFile = new CachedFile<>(tmpFile, transformer, refreshPolicy); + + assertEquals(cachedFile.lastModified(), tmpFile.lastModified()); + assertEquals(7, cachedFile.size()); + assertEquals(" foo ", cachedFile.contents()); + assertEquals("foo", cachedFile.transformed()); + + // Sleep for a bit to make sure our timestamp changes, then update the file. + Utils.sleep(10); + Files.writeString(tmpFile.toPath(), " bar baz ", StandardOpenOption.WRITE, StandardOpenOption.APPEND); + + assertNotEquals(cachedFile.lastModified(), tmpFile.lastModified()); + assertNotEquals(cachedFile.size(), tmpFile.length()); + assertEquals(7, cachedFile.size()); + assertEquals(" foo ", cachedFile.contents()); + assertEquals("foo", cachedFile.transformed()); + } + + @Test + public void testLastModifiedPolicy() throws Exception { + File tmpFile = tempFile(" foo "); + + CachedFile.Transformer transformer = (file, contents) -> contents.trim(); + CachedFile.RefreshPolicy refreshPolicy = CachedFile.RefreshPolicy.lastModifiedPolicy(); + CachedFile cachedFile = new CachedFile<>(tmpFile, transformer, refreshPolicy); + + assertEquals(cachedFile.lastModified(), tmpFile.lastModified()); + assertEquals(7, cachedFile.size()); + assertEquals(" foo ", cachedFile.contents()); + assertEquals("foo", cachedFile.transformed()); + + // Sleep for a bit to make sure our timestamp changes, then update the file. + Utils.sleep(10); + Files.writeString(tmpFile.toPath(), " bar baz ", StandardOpenOption.WRITE, StandardOpenOption.APPEND); + + assertEquals(18, cachedFile.size()); + assertEquals(" foo bar baz ", cachedFile.contents()); + assertEquals("foo bar baz", cachedFile.transformed()); + } + + @Test + public void testFileDoesNotExist() throws IOException { + File tmpFile = tempFile(" foo "); + + CachedFile.RefreshPolicy refreshPolicy = CachedFile.RefreshPolicy.lastModifiedPolicy(); + CachedFile cachedFile = new CachedFile<>(tmpFile, CachedFile.STRING_NOOP_TRANSFORMER, refreshPolicy); + + // All is well... + assertTrue(tmpFile.exists()); + assertDoesNotThrow(cachedFile::size); + assertDoesNotThrow(cachedFile::lastModified); + assertDoesNotThrow(cachedFile::contents); + assertDoesNotThrow(cachedFile::transformed); + + // Delete the file and ensure that exceptions are thrown + assertTrue(tmpFile.delete()); + Utils.sleep(50); + + assertFalse(tmpFile.exists()); + assertThrows(KafkaException.class, cachedFile::size); + assertThrows(KafkaException.class, cachedFile::lastModified); + assertThrows(KafkaException.class, cachedFile::contents); + assertThrows(KafkaException.class, cachedFile::transformed); + + System.out.println("yo"); + + // "Restore" the file and make sure it's refreshed. + Utils.sleep(10); + Files.writeString(tmpFile.toPath(), "valid data!", StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW); + + assertTrue(tmpFile.exists()); + assertDoesNotThrow(cachedFile::size); + assertDoesNotThrow(cachedFile::lastModified); + assertDoesNotThrow(cachedFile::contents); + assertDoesNotThrow(cachedFile::transformed); + } + + @Test + public void testTransformerError() throws Exception { + File tmpFile = tempFile("[\"foo\"]"); + + @SuppressWarnings("unchecked") + CachedFile.Transformer> jsonTransformer = (file, json) -> { + try { + ObjectMapper mapper = new ObjectMapper(); + return (List) mapper.readValue(json, List.class); + } catch (Exception e) { + throw new KafkaException(e); + } + }; + + CachedFile.RefreshPolicy> refreshPolicy = CachedFile.RefreshPolicy.lastModifiedPolicy(); + CachedFile> cachedFile = new CachedFile<>(tmpFile, jsonTransformer, refreshPolicy); + + assertEquals(List.of("foo"), cachedFile.transformed()); + + // Sleep then update the file with proper JSON. + Utils.sleep(10); + Files.writeString(tmpFile.toPath(), "[\"foo\", \"bar\", \"baz\"]", StandardOpenOption.WRITE, StandardOpenOption.TRUNCATE_EXISTING); + + assertEquals(List.of("foo", "bar", "baz"), cachedFile.transformed()); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientCredentialsRequestFormatterTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientCredentialsRequestFormatterTest.java new file mode 100644 index 00000000000..885abc56928 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientCredentialsRequestFormatterTest.java @@ -0,0 +1,141 @@ +/* + * 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.kafka.common.security.oauthbearer.internals.secured; + +import org.apache.kafka.common.config.ConfigException; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.stream.Stream; + +import static org.apache.kafka.common.security.oauthbearer.internals.secured.ClientCredentialsRequestFormatter.GRANT_TYPE; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class ClientCredentialsRequestFormatterTest extends OAuthBearerTest { + + public static final String CLIENT_ID = "jdoe"; + public static final String CLIENT_SECRET = "secret"; + public static final String SCOPE = "everythingeverything"; + + @Test + public void testFormatAuthorizationHeaderEncoding() { + // according to RFC-7617, we need to use the *non-URL safe* base64 encoder. See KAFKA-14496. + assertAuthorizationHeaderEquals("SOME_RANDOM_LONG_USER_01234", "9Q|0`8i~ute-n9ksjLWb\\50\"AX@UUED5E", false, "Basic U09NRV9SQU5ET01fTE9OR19VU0VSXzAxMjM0OjlRfDBgOGl+dXRlLW45a3NqTFdiXDUwIkFYQFVVRUQ1RQ=="); + // according to RFC-6749 clientId & clientSecret must be urlencoded, see https://tools.ietf.org/html/rfc6749#section-2.3.1 + assertAuthorizationHeaderEquals("user!@~'", "secret-(*)!", true, "Basic dXNlciUyMSU0MCU3RSUyNzpzZWNyZXQtJTI4KiUyOSUyMQ=="); + } + + @ParameterizedTest + @MethodSource("testFormatterMissingValuesSource") + public void testFormatterMissingValues(String clientId, String clientSecret, boolean urlencode) { + assertThrows( + ConfigException.class, + () -> new ClientCredentialsRequestFormatter( + clientId, + clientSecret, + SCOPE, + urlencode + ) + ); + } + + @ParameterizedTest + @MethodSource("testScopeEscapingSource") + public void testScopeEscaping(String scope, boolean urlencode, String expectedScope) { + String expected = "grant_type=" + GRANT_TYPE + "&scope=" + expectedScope; + assertRequestBodyEquals(scope, urlencode, expected); + } + + @ParameterizedTest + @MethodSource("testMissingScopesSource") + public void testMissingScopes(String scope, boolean urlencode) { + String expected = "grant_type=" + GRANT_TYPE; + assertRequestBodyEquals(scope, urlencode, expected); + } + + private static Stream testFormatterMissingValuesSource() { + String[] clientIds = new String[] {null, "", " ", CLIENT_ID}; + String[] clientSecrets = new String[] {null, "", " ", CLIENT_SECRET}; + boolean[] urlencodes = new boolean[] {true, false}; + + List list = new ArrayList<>(); + + for (String clientId : clientIds) { + for (String clientSecret : clientSecrets) { + for (boolean urlencode : urlencodes) { + if (CLIENT_ID.equals(clientId) && CLIENT_SECRET.equals(clientSecret)) + continue; + + list.add(Arguments.of(clientId, clientSecret, urlencode)); + } + } + } + + return list.stream(); + } + + private static Stream testMissingScopesSource() { + String[] scopes = new String[] {null, "", " "}; + boolean[] urlencodes = new boolean[] {true, false}; + + List list = new ArrayList<>(); + + for (String scope : scopes) { + for (boolean urlencode : urlencodes) { + list.add(Arguments.of(scope, urlencode)); + } + } + + return list.stream(); + } + + private static Stream testScopeEscapingSource() { + return Stream.of( + Arguments.of("test-scope", true, "test-scope"), + Arguments.of("test-scope", false, "test-scope"), + Arguments.of("earth is great!", true, "earth+is+great%21"), + Arguments.of("earth is great!", false, "earth is great!"), + Arguments.of("what on earth?!?!?", true, "what+on+earth%3F%21%3F%21%3F"), + Arguments.of("what on earth?!?!?", false, "what on earth?!?!?") + ); + } + + private void assertRequestBodyEquals(String scope, boolean urlencode, String expected) { + ClientCredentialsRequestFormatter formatter = new ClientCredentialsRequestFormatter( + CLIENT_ID, + CLIENT_SECRET, + scope, + urlencode + ); + String actual = formatter.formatBody(); + assertEquals(expected, actual); + } + + private void assertAuthorizationHeaderEquals(String clientId, String clientSecret, boolean urlencode, String expected) { + ClientCredentialsRequestFormatter formatter = new ClientCredentialsRequestFormatter(clientId, clientSecret, SCOPE, urlencode); + Map headers = formatter.formatHeaders(); + String actual = headers.get("Authorization"); + assertEquals(expected, actual, String.format("Expected the HTTP Authorization header generated for client ID \"%s\" and client secret \"%s\" to match", clientId, clientSecret)); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtilsTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtilsTest.java index 9a62f480215..efc41d64b32 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtilsTest.java @@ -26,16 +26,16 @@ import org.junit.jupiter.api.Test; import java.io.File; import java.io.IOException; import java.util.Collections; -import java.util.HashMap; import java.util.Map; +import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_FILES_CONFIG; import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; public class ConfigurationUtilsTest extends OAuthBearerTest { - private static final String URL_CONFIG_NAME = "url"; - private static final String FILE_CONFIG_NAME = "file"; + private static final String URL_CONFIG_NAME = "fictitious.url.config"; + private static final String FILE_CONFIG_NAME = "fictitious.file.config"; @AfterEach public void tearDown() throws Exception { @@ -59,7 +59,7 @@ public class ConfigurationUtilsTest extends OAuthBearerTest { @Test public void testUrlFile() { - testUrl("file:///tmp/foo.txt"); + assertThrowsWithMessage(ConfigException.class, () -> testFileUrl("file:///tmp/foo.txt"), "that doesn't exist"); } @Test @@ -74,41 +74,34 @@ public class ConfigurationUtilsTest extends OAuthBearerTest { @Test public void testUrlInvalidProtocol() { - assertThrowsWithMessage(ConfigException.class, () -> testUrl("ftp://ftp.example.com"), "invalid protocol"); + assertThrowsWithMessage(ConfigException.class, () -> testFileUrl("ftp://ftp.example.com"), "invalid protocol"); } @Test public void testUrlNull() { - assertThrowsWithMessage(ConfigException.class, () -> testUrl(null), "must be non-null"); + assertThrowsWithMessage(ConfigException.class, () -> testUrl(null), "is required"); } @Test public void testUrlEmptyString() { - assertThrowsWithMessage(ConfigException.class, () -> testUrl(""), "must not contain only whitespace"); + assertThrowsWithMessage(ConfigException.class, () -> testUrl(""), "is required"); } @Test public void testUrlWhitespace() { - assertThrowsWithMessage(ConfigException.class, () -> testUrl(" "), "must not contain only whitespace"); - } - - private void testUrl(String value) { - System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, value == null ? "" : value); - Map configs = Collections.singletonMap(URL_CONFIG_NAME, value); - ConfigurationUtils cu = new ConfigurationUtils(configs); - cu.validateUrl(URL_CONFIG_NAME); + assertThrowsWithMessage(ConfigException.class, () -> testUrl(" "), "is required"); } @Test public void testFile() throws IOException { File file = TestUtils.tempFile("some contents!"); - testFile(file.toURI().toURL().toString()); + testFile(file.getAbsolutePath()); } @Test public void testFileWithSuperfluousWhitespace() throws IOException { File file = TestUtils.tempFile(); - testFile(String.format(" %s ", file.toURI().toURL())); + testFile(String.format(" %s ", file.getAbsolutePath())); } @Test @@ -123,56 +116,90 @@ public class ConfigurationUtilsTest extends OAuthBearerTest { if (!file.setReadable(false)) throw new IllegalStateException(String.format("Can't test file permissions as test couldn't programmatically make temp file %s un-readable", file.getAbsolutePath())); - assertThrowsWithMessage(ConfigException.class, () -> testFile(file.toURI().toURL().toString()), "that doesn't have read permission"); + assertThrowsWithMessage(ConfigException.class, () -> testFile(file.getAbsolutePath()), "that doesn't have read permission"); } @Test public void testFileNull() { - assertThrowsWithMessage(ConfigException.class, () -> testFile(null), "must be non-null"); + assertThrowsWithMessage(ConfigException.class, () -> testFile(null), "is required"); } @Test public void testFileEmptyString() { - assertThrowsWithMessage(ConfigException.class, () -> testFile(""), "must not contain only whitespace"); + assertThrowsWithMessage(ConfigException.class, () -> testFile(""), "is required"); } @Test public void testFileWhitespace() { - assertThrowsWithMessage(ConfigException.class, () -> testFile(" "), "must not contain only whitespace"); + assertThrowsWithMessage(ConfigException.class, () -> testFile(" "), "is required"); } @Test public void testThrowIfURLIsNotAllowed() { String url = "http://www.example.com"; String fileUrl = "file:///etc/passwd"; - Map configs = new HashMap<>(); - configs.put(URL_CONFIG_NAME, url); - configs.put(FILE_CONFIG_NAME, fileUrl); - ConfigurationUtils cu = new ConfigurationUtils(configs); + ConfigurationUtils cu = new ConfigurationUtils(Map.of()); // By default, no URL is allowed - assertThrowsWithMessage(ConfigException.class, () -> cu.throwIfURLIsNotAllowed(url), + assertThrowsWithMessage(ConfigException.class, () -> cu.throwIfURLIsNotAllowed(URL_CONFIG_NAME, url), ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG); - assertThrowsWithMessage(ConfigException.class, () -> cu.throwIfURLIsNotAllowed(fileUrl), + assertThrowsWithMessage(ConfigException.class, () -> cu.throwIfURLIsNotAllowed(FILE_CONFIG_NAME, fileUrl), ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG); // add one url into allowed list System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, url); - assertDoesNotThrow(() -> cu.throwIfURLIsNotAllowed(url)); - assertThrowsWithMessage(ConfigException.class, () -> cu.throwIfURLIsNotAllowed(fileUrl), + assertDoesNotThrow(() -> cu.throwIfURLIsNotAllowed(URL_CONFIG_NAME, url)); + assertThrowsWithMessage(ConfigException.class, () -> cu.throwIfURLIsNotAllowed(FILE_CONFIG_NAME, fileUrl), ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG); // add all urls into allowed list System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, url + "," + fileUrl); - assertDoesNotThrow(() -> cu.throwIfURLIsNotAllowed(url)); - assertDoesNotThrow(() -> cu.throwIfURLIsNotAllowed(fileUrl)); + assertDoesNotThrow(() -> cu.throwIfURLIsNotAllowed(URL_CONFIG_NAME, url)); + assertDoesNotThrow(() -> cu.throwIfURLIsNotAllowed(FILE_CONFIG_NAME, fileUrl)); } - protected void testFile(String value) { + @Test + public void testThrowIfFileIsNotAllowed() { + String file1 = "file1"; + String file2 = "file2"; + ConfigurationUtils cu = new ConfigurationUtils(Map.of()); + + // By default, no file is allowed + assertThrowsWithMessage(ConfigException.class, () -> cu.throwIfFileIsNotAllowed(FILE_CONFIG_NAME, file1), + ALLOWED_SASL_OAUTHBEARER_FILES_CONFIG); + assertThrowsWithMessage(ConfigException.class, () -> cu.throwIfFileIsNotAllowed(FILE_CONFIG_NAME, file1), + ALLOWED_SASL_OAUTHBEARER_FILES_CONFIG); + + // add one file into allowed list + System.setProperty(ALLOWED_SASL_OAUTHBEARER_FILES_CONFIG, file1); + assertDoesNotThrow(() -> cu.throwIfFileIsNotAllowed(FILE_CONFIG_NAME, file1)); + assertThrowsWithMessage(ConfigException.class, () -> cu.throwIfFileIsNotAllowed(FILE_CONFIG_NAME, file2), + ALLOWED_SASL_OAUTHBEARER_FILES_CONFIG); + + // add all files into allowed list + System.setProperty(ALLOWED_SASL_OAUTHBEARER_FILES_CONFIG, file1 + "," + file2); + assertDoesNotThrow(() -> cu.throwIfFileIsNotAllowed(FILE_CONFIG_NAME, file1)); + assertDoesNotThrow(() -> cu.throwIfFileIsNotAllowed(FILE_CONFIG_NAME, file2)); + } + + private void testUrl(String value) { System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, value == null ? "" : value); Map configs = Collections.singletonMap(URL_CONFIG_NAME, value); ConfigurationUtils cu = new ConfigurationUtils(configs); - cu.validateFile(URL_CONFIG_NAME); + cu.validateUrl(URL_CONFIG_NAME); } + private void testFile(String value) { + System.setProperty(ALLOWED_SASL_OAUTHBEARER_FILES_CONFIG, value == null ? "" : value); + Map configs = Collections.singletonMap(FILE_CONFIG_NAME, value); + ConfigurationUtils cu = new ConfigurationUtils(configs); + cu.validateFile(FILE_CONFIG_NAME); + } + + private void testFileUrl(String value) { + System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, value == null ? "" : value); + Map configs = Collections.singletonMap(URL_CONFIG_NAME, value); + ConfigurationUtils cu = new ConfigurationUtils(configs); + cu.validateFileUrl(URL_CONFIG_NAME); + } } diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpJwtRetrieverTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpJwtRetrieverTest.java index 0bd903300ff..7a6835894c1 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpJwtRetrieverTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpJwtRetrieverTest.java @@ -17,9 +17,6 @@ package org.apache.kafka.common.security.oauthbearer.internals.secured; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.node.ObjectNode; - import org.junit.jupiter.api.Test; import java.io.ByteArrayInputStream; @@ -135,104 +132,4 @@ public class HttpJwtRetrieverTest extends OAuthBearerTest { when(mockedIn.read(any(byte[].class))).thenThrow(new IOException()); assertThrows(IOException.class, () -> HttpJwtRetriever.copy(mockedIn, out)); } - - @Test - public void testParseAccessToken() throws IOException { - String expected = "abc"; - ObjectMapper mapper = new ObjectMapper(); - ObjectNode node = mapper.createObjectNode(); - node.put("access_token", expected); - - String actual = HttpJwtRetriever.parseAccessToken(mapper.writeValueAsString(node)); - assertEquals(expected, actual); - } - - @Test - public void testParseAccessTokenEmptyAccessToken() { - ObjectMapper mapper = new ObjectMapper(); - ObjectNode node = mapper.createObjectNode(); - node.put("access_token", ""); - - assertThrows(IllegalArgumentException.class, () -> HttpJwtRetriever.parseAccessToken(mapper.writeValueAsString(node))); - } - - @Test - public void testParseAccessTokenMissingAccessToken() { - ObjectMapper mapper = new ObjectMapper(); - ObjectNode node = mapper.createObjectNode(); - node.put("sub", "jdoe"); - - assertThrows(IllegalArgumentException.class, () -> HttpJwtRetriever.parseAccessToken(mapper.writeValueAsString(node))); - } - - @Test - public void testParseAccessTokenInvalidJson() { - assertThrows(IOException.class, () -> HttpJwtRetriever.parseAccessToken("not valid JSON")); - } - - @Test - public void testFormatAuthorizationHeader() { - assertAuthorizationHeader("id", "secret", false, "Basic aWQ6c2VjcmV0"); - } - - @Test - public void testFormatAuthorizationHeaderEncoding() { - // according to RFC-7617, we need to use the *non-URL safe* base64 encoder. See KAFKA-14496. - assertAuthorizationHeader("SOME_RANDOM_LONG_USER_01234", "9Q|0`8i~ute-n9ksjLWb\\50\"AX@UUED5E", false, "Basic U09NRV9SQU5ET01fTE9OR19VU0VSXzAxMjM0OjlRfDBgOGl+dXRlLW45a3NqTFdiXDUwIkFYQFVVRUQ1RQ=="); - // according to RFC-6749 clientId & clientSecret must be urlencoded, see https://tools.ietf.org/html/rfc6749#section-2.3.1 - assertAuthorizationHeader("user!@~'", "secret-(*)!", true, "Basic dXNlciUyMSU0MCU3RSUyNzpzZWNyZXQtJTI4KiUyOSUyMQ=="); - } - - private void assertAuthorizationHeader(String clientId, String clientSecret, boolean urlencode, String expected) { - String actual = HttpJwtRetriever.formatAuthorizationHeader(clientId, clientSecret, urlencode); - assertEquals(expected, actual, String.format("Expected the HTTP Authorization header generated for client ID \"%s\" and client secret \"%s\" to match", clientId, clientSecret)); - } - - @Test - public void testFormatAuthorizationHeaderMissingValues() { - assertThrows(IllegalArgumentException.class, () -> HttpJwtRetriever.formatAuthorizationHeader(null, "secret", false)); - assertThrows(IllegalArgumentException.class, () -> HttpJwtRetriever.formatAuthorizationHeader("id", null, false)); - assertThrows(IllegalArgumentException.class, () -> HttpJwtRetriever.formatAuthorizationHeader(null, null, false)); - assertThrows(IllegalArgumentException.class, () -> HttpJwtRetriever.formatAuthorizationHeader("", "secret", false)); - assertThrows(IllegalArgumentException.class, () -> HttpJwtRetriever.formatAuthorizationHeader("id", "", false)); - assertThrows(IllegalArgumentException.class, () -> HttpJwtRetriever.formatAuthorizationHeader("", "", false)); - assertThrows(IllegalArgumentException.class, () -> HttpJwtRetriever.formatAuthorizationHeader(" ", "secret", false)); - assertThrows(IllegalArgumentException.class, () -> HttpJwtRetriever.formatAuthorizationHeader("id", " ", false)); - assertThrows(IllegalArgumentException.class, () -> HttpJwtRetriever.formatAuthorizationHeader(" ", " ", false)); - } - - @Test - public void testFormatRequestBody() { - String expected = "grant_type=client_credentials&scope=scope"; - String actual = HttpJwtRetriever.formatRequestBody("scope"); - assertEquals(expected, actual); - } - - @Test - public void testFormatRequestBodyWithEscaped() { - String questionMark = "%3F"; - String exclamationMark = "%21"; - - String expected = String.format("grant_type=client_credentials&scope=earth+is+great%s", exclamationMark); - String actual = HttpJwtRetriever.formatRequestBody("earth is great!"); - assertEquals(expected, actual); - - expected = String.format("grant_type=client_credentials&scope=what+on+earth%s%s%s%s%s", questionMark, exclamationMark, questionMark, exclamationMark, questionMark); - actual = HttpJwtRetriever.formatRequestBody("what on earth?!?!?"); - assertEquals(expected, actual); - } - - @Test - public void testFormatRequestBodyMissingValues() { - String expected = "grant_type=client_credentials"; - String actual = HttpJwtRetriever.formatRequestBody(null); - assertEquals(expected, actual); - - actual = HttpJwtRetriever.formatRequestBody(""); - assertEquals(expected, actual); - - actual = HttpJwtRetriever.formatRequestBody(" "); - assertEquals(expected, actual); - } - } diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtResponseParserTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtResponseParserTest.java new file mode 100644 index 00000000000..c175cbcb945 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtResponseParserTest.java @@ -0,0 +1,71 @@ +/* + * 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.kafka.common.security.oauthbearer.internals.secured; + +import org.apache.kafka.common.security.oauthbearer.JwtRetrieverException; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; + +import org.junit.jupiter.api.Test; + +import java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class JwtResponseParserTest extends OAuthBearerTest { + + @Test + public void testParseJwt() throws IOException { + String expected = "abc"; + ObjectMapper mapper = new ObjectMapper(); + ObjectNode node = mapper.createObjectNode(); + node.put("access_token", expected); + + JwtResponseParser responseParser = new JwtResponseParser(); + String actual = responseParser.parseJwt(mapper.writeValueAsString(node)); + assertEquals(expected, actual); + } + + @Test + public void testParseJwtEmptyAccessToken() { + ObjectMapper mapper = new ObjectMapper(); + ObjectNode node = mapper.createObjectNode(); + node.put("access_token", ""); + + JwtResponseParser responseParser = new JwtResponseParser(); + assertThrows(JwtRetrieverException.class, () -> responseParser.parseJwt(mapper.writeValueAsString(node))); + } + + @Test + public void testParseJwtMissingAccessToken() { + ObjectMapper mapper = new ObjectMapper(); + ObjectNode node = mapper.createObjectNode(); + node.put("sub", "jdoe"); + + JwtResponseParser responseParser = new JwtResponseParser(); + assertThrows(JwtRetrieverException.class, () -> responseParser.parseJwt(mapper.writeValueAsString(node))); + } + + @Test + public void testParseJwtInvalidJson() { + JwtResponseParser responseParser = new JwtResponseParser(); + assertThrows(JwtRetrieverException.class, () -> responseParser.parseJwt("not valid JSON")); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java index 8e82092f28d..7a0aeea3d3d 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java @@ -19,6 +19,8 @@ package org.apache.kafka.common.security.oauthbearer.internals.secured; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import com.fasterxml.jackson.databind.ObjectMapper; @@ -27,28 +29,41 @@ import com.fasterxml.jackson.databind.node.ObjectNode; import org.jose4j.jwk.PublicJsonWebKey; import org.jose4j.jwk.RsaJsonWebKey; import org.jose4j.jwk.RsaJwkGenerator; +import org.jose4j.jwt.consumer.InvalidJwtException; +import org.jose4j.jwt.consumer.JwtConsumer; +import org.jose4j.jwt.consumer.JwtConsumerBuilder; +import org.jose4j.jwt.consumer.JwtContext; import org.jose4j.lang.JoseException; import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.TestInstance.Lifecycle; import org.junit.jupiter.api.function.Executable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.File; -import java.io.FileWriter; import java.io.IOException; import java.net.HttpURLConnection; import java.net.URL; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.file.StandardOpenOption; +import java.security.KeyPair; +import java.security.KeyPairGenerator; +import java.security.NoSuchAlgorithmException; +import java.security.PrivateKey; +import java.security.PublicKey; import java.util.Arrays; import java.util.Base64; import java.util.Collections; +import java.util.EnumSet; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.concurrent.ExecutionException; import java.util.function.Consumer; +import javax.security.auth.login.AppConfigurationEntry; + import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; @@ -58,8 +73,6 @@ import static org.mockito.Mockito.when; @TestInstance(Lifecycle.PER_CLASS) public abstract class OAuthBearerTest { - protected final Logger log = LoggerFactory.getLogger(getClass()); - protected ObjectMapper mapper = new ObjectMapper(); protected void assertThrowsWithMessage(Class clazz, @@ -130,36 +143,6 @@ public abstract class OAuthBearerTest { return mockedCon; } - protected File createTempDir(String directory) throws IOException { - File tmpDir = new File(System.getProperty("java.io.tmpdir")); - - if (directory != null) - tmpDir = new File(tmpDir, directory); - - if (!tmpDir.exists() && !tmpDir.mkdirs()) - throw new IOException("Could not create " + tmpDir); - - tmpDir.deleteOnExit(); - log.debug("Created temp directory {}", tmpDir); - return tmpDir; - } - - protected File createTempFile(File tmpDir, - String prefix, - String suffix, - String contents) - throws IOException { - File file = File.createTempFile(prefix, suffix, tmpDir); - log.debug("Created new temp file {}", file); - file.deleteOnExit(); - - try (FileWriter writer = new FileWriter(file)) { - writer.write(contents); - } - - return file; - } - protected Map getSaslConfigs(Map configs) { ConfigDef configDef = new ConfigDef(); configDef.withClientSaslSupport(); @@ -175,6 +158,20 @@ public abstract class OAuthBearerTest { return getSaslConfigs(Collections.emptyMap()); } + protected List getJaasConfigEntries() { + return getJaasConfigEntries(Map.of()); + } + + protected List getJaasConfigEntries(Map options) { + return List.of( + new AppConfigurationEntry( + OAuthBearerLoginModule.class.getName(), + AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, + options + ) + ); + } + protected PublicJsonWebKey createRsaJwk() throws JoseException { RsaJsonWebKey jwk = RsaJwkGenerator.generateJwk(2048); jwk.setKeyId("key-1"); @@ -195,11 +192,75 @@ public abstract class OAuthBearerTest { return jwk; } - protected String createAccessKey(String header, String payload, String signature) { + protected String createJwt(String header, String payload, String signature) { Base64.Encoder enc = Base64.getEncoder(); header = enc.encodeToString(Utils.utf8(header)); payload = enc.encodeToString(Utils.utf8(payload)); signature = enc.encodeToString(Utils.utf8(signature)); return String.format("%s.%s.%s", header, payload, signature); } + + protected String createJwt(String subject) { + Time time = Time.SYSTEM; + long nowSeconds = time.milliseconds() / 1000; + + return createJwt( + "{}", + String.format( + "{\"iat\":%s, \"exp\":%s, \"sub\":\"%s\"}", + nowSeconds, + nowSeconds + 300, + subject + ), + "sign" + ); + } + + + protected void assertClaims(PublicKey publicKey, String assertion) throws InvalidJwtException { + JwtConsumer jwtConsumer = jwtConsumer(publicKey); + jwtConsumer.processToClaims(assertion); + } + + protected JwtContext assertContext(PublicKey publicKey, String assertion) throws InvalidJwtException { + JwtConsumer jwtConsumer = jwtConsumer(publicKey); + return jwtConsumer.process(assertion); + } + + protected JwtConsumer jwtConsumer(PublicKey publicKey) { + return new JwtConsumerBuilder() + .setVerificationKey(publicKey) + .setRequireExpirationTime() + .setAllowedClockSkewInSeconds(30) // Sure, let's give it some slack + .build(); + } + + protected File generatePrivateKey(PrivateKey privateKey) throws IOException { + File file = File.createTempFile("private-", ".key"); + byte[] bytes = Base64.getEncoder().encode(privateKey.getEncoded()); + + try (FileChannel channel = FileChannel.open(file.toPath(), EnumSet.of(StandardOpenOption.WRITE))) { + Utils.writeFully(channel, ByteBuffer.wrap(bytes)); + } + + return file; + } + + protected File generatePrivateKey() throws IOException { + return generatePrivateKey(generateKeyPair().getPrivate()); + } + + protected KeyPair generateKeyPair() { + return generateKeyPair("RSA"); + } + + protected KeyPair generateKeyPair(String algorithm) { + try { + KeyPairGenerator keyGen = KeyPairGenerator.getInstance(algorithm); + keyGen.initialize(2048); + return keyGen.generateKeyPair(); + } catch (NoSuchAlgorithmException e) { + throw new IllegalStateException("Received unexpected error during private key generation", e); + } + } } \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/VerificationKeyResolverFactoryTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/VerificationKeyResolverFactoryTest.java index c2324b9d2da..b515255147f 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/VerificationKeyResolverFactoryTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/VerificationKeyResolverFactoryTest.java @@ -28,6 +28,8 @@ import java.util.Map; import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL; import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG; +import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM; +import static org.apache.kafka.test.TestUtils.tempFile; public class VerificationKeyResolverFactoryTest extends OAuthBearerTest { @@ -38,15 +40,10 @@ public class VerificationKeyResolverFactoryTest extends OAuthBearerTest { @Test public void testConfigureRefreshingFileVerificationKeyResolver() throws Exception { - File tmpDir = createTempDir("access-token"); - File verificationKeyFile = createTempFile(tmpDir, "access-token-", ".json", "{}"); - - System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, verificationKeyFile.toURI().toString()); - Map configs = Collections.singletonMap(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL, verificationKeyFile.toURI().toString()); - Map jaasConfig = Collections.emptyMap(); - - // verify it won't throw exception - try (CloseableVerificationKeyResolver verificationKeyResolver = VerificationKeyResolverFactory.create(configs, jaasConfig)) { } + String file = tempFile("{}").toURI().toString(); + System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, file); + Map configs = Collections.singletonMap(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL, file); + assertThrowsWithMessage(ConfigException.class, () -> VerificationKeyResolverFactory.create(configs, OAUTHBEARER_MECHANISM, getJaasConfigEntries()), "The JSON JWKS content does not include the keys member"); } @Test @@ -55,28 +52,15 @@ public class VerificationKeyResolverFactoryTest extends OAuthBearerTest { String file = new File("/tmp/this-directory-does-not-exist/foo.json").toURI().toString(); System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, file); Map configs = getSaslConfigs(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL, file); - Map jaasConfig = Collections.emptyMap(); - assertThrowsWithMessage(ConfigException.class, () -> VerificationKeyResolverFactory.create(configs, jaasConfig), "that doesn't exist"); - } - - @Test - public void testConfigureRefreshingFileVerificationKeyResolverWithInvalidFile() throws Exception { - // Should fail because while the parent path exists, the file itself doesn't. - File tmpDir = createTempDir("this-directory-does-exist"); - File verificationKeyFile = new File(tmpDir, "this-file-does-not-exist.json"); - System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, verificationKeyFile.toURI().toString()); - Map configs = getSaslConfigs(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL, verificationKeyFile.toURI().toString()); - Map jaasConfig = Collections.emptyMap(); - assertThrowsWithMessage(ConfigException.class, () -> VerificationKeyResolverFactory.create(configs, jaasConfig), "that doesn't exist"); + assertThrowsWithMessage(ConfigException.class, () -> VerificationKeyResolverFactory.create(configs, OAUTHBEARER_MECHANISM, getJaasConfigEntries()), "that doesn't exist"); } @Test public void testSaslOauthbearerTokenEndpointUrlIsNotAllowed() throws Exception { // Should fail if the URL is not allowed - File tmpDir = createTempDir("not_allowed"); - File verificationKeyFile = new File(tmpDir, "not_allowed.json"); - Map configs = getSaslConfigs(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL, verificationKeyFile.toURI().toString()); - assertThrowsWithMessage(ConfigException.class, () -> VerificationKeyResolverFactory.create(configs, Collections.emptyMap()), + String file = tempFile("{}").toURI().toString(); + Map configs = getSaslConfigs(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL, file); + assertThrowsWithMessage(ConfigException.class, () -> VerificationKeyResolverFactory.create(configs, OAUTHBEARER_MECHANISM, getJaasConfigEntries()), ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG); } } diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/DefaultAssertionCreatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/DefaultAssertionCreatorTest.java new file mode 100644 index 00000000000..d5b165b4688 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/DefaultAssertionCreatorTest.java @@ -0,0 +1,193 @@ +/* + * 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.kafka.common.security.oauthbearer.internals.secured.assertion; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerTest; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; + +import org.jose4j.jwt.consumer.JwtContext; +import org.jose4j.jwx.JsonWebStructure; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; + +import java.io.File; +import java.io.IOException; +import java.nio.channels.FileChannel; +import java.nio.file.StandardOpenOption; +import java.security.GeneralSecurityException; +import java.security.KeyPair; +import java.security.NoSuchAlgorithmException; +import java.security.PrivateKey; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.apache.kafka.common.security.oauthbearer.internals.secured.assertion.AssertionUtils.TOKEN_SIGNING_ALGORITHM_RS256; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.assertion.AssertionUtils.getSignature; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.assertion.AssertionUtils.sign; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class DefaultAssertionCreatorTest extends OAuthBearerTest { + + @Test + public void testPrivateKey() throws Exception { + KeyPair keyPair = generateKeyPair(); + Builder builder = new Builder() + .setPrivateKeyFile(generatePrivateKey(keyPair.getPrivate())); + AssertionJwtTemplate jwtTemplate = new LayeredAssertionJwtTemplate( + new StaticAssertionJwtTemplate(Map.of("kid", "test-id"), Map.of()), + new DynamicAssertionJwtTemplate( + new MockTime(), + builder.algorithm, + 3600, + 60, + false + ) + ); + + try (AssertionCreator assertionCreator = builder.build()) { + String assertion = assertionCreator.create(jwtTemplate); + assertClaims(keyPair.getPublic(), assertion); + } + } + + @Test + public void testPrivateKeyId() throws Exception { + KeyPair keyPair = generateKeyPair(); + Builder builder = new Builder() + .setPrivateKeyFile(generatePrivateKey(keyPair.getPrivate())); + + AssertionJwtTemplate jwtTemplate = new LayeredAssertionJwtTemplate( + new StaticAssertionJwtTemplate(Map.of("kid", "test-id"), Map.of()), + new DynamicAssertionJwtTemplate( + new MockTime(), + builder.algorithm, + 3600, + 60, + false + ) + ); + + try (AssertionCreator assertionCreator = builder.build()) { + String assertion = assertionCreator.create(jwtTemplate); + JwtContext context = assertContext(keyPair.getPublic(), assertion); + List joseObjects = context.getJoseObjects(); + assertNotNull(joseObjects); + assertEquals(1, joseObjects.size()); + JsonWebStructure jsonWebStructure = joseObjects.get(0); + assertEquals("test-id", jsonWebStructure.getKeyIdHeaderValue()); + } + } + + @Test + public void testInvalidPrivateKey() throws Exception { + File privateKeyFile = generatePrivateKey(); + long originalFileLength = privateKeyFile.length(); + int bytesToTruncate = 10; // A single byte isn't enough + + // Intentionally "mangle" the private key secret by truncating the file. + try (FileChannel channel = FileChannel.open(privateKeyFile.toPath(), StandardOpenOption.WRITE)) { + long size = channel.size(); + assertEquals(originalFileLength, size); + assertTrue(size > bytesToTruncate); + channel.truncate(size - bytesToTruncate); + } + + assertEquals(originalFileLength - bytesToTruncate, privateKeyFile.length()); + + KafkaException e = assertThrows(KafkaException.class, () -> new Builder().setPrivateKeyFile(privateKeyFile).build()); + assertNotNull(e.getCause()); + assertInstanceOf(GeneralSecurityException.class, e.getCause()); + } + + @ParameterizedTest + @CsvSource("RS256,ES256") + public void testAlgorithm(String algorithm) throws Exception { + KeyPair keyPair = generateKeyPair(); + Builder builder = new Builder() + .setPrivateKeyFile(generatePrivateKey(keyPair.getPrivate())) + .setAlgorithm(algorithm); + + String assertion; + + try (AssertionCreator assertionCreator = builder.build()) { + AssertionJwtTemplate jwtTemplate = new DynamicAssertionJwtTemplate( + new MockTime(), + algorithm, + 3600, + 60, + false + ); + assertion = assertionCreator.create(jwtTemplate); + } + + assertClaims(keyPair.getPublic(), assertion); + + JwtContext context = assertContext(keyPair.getPublic(), assertion); + List joseObjects = context.getJoseObjects(); + assertNotNull(joseObjects); + assertEquals(1, joseObjects.size()); + JsonWebStructure jsonWebStructure = joseObjects.get(0); + assertEquals(algorithm, jsonWebStructure.getAlgorithmHeaderValue()); + } + + @Test + public void testInvalidAlgorithm() throws IOException { + PrivateKey privateKey = generateKeyPair().getPrivate(); + Builder builder = new Builder() + .setPrivateKeyFile(generatePrivateKey(privateKey)) + .setAlgorithm("thisisnotvalid"); + assertThrows(NoSuchAlgorithmException.class, () -> getSignature(builder.algorithm)); + assertThrows( + NoSuchAlgorithmException.class, + () -> sign(builder.algorithm, privateKey, "dummy content")); + } + + private static class Builder { + + private final Time time = new MockTime(); + private String algorithm = TOKEN_SIGNING_ALGORITHM_RS256; + private File privateKeyFile; + private Optional passphrase = Optional.empty(); + + public Builder setAlgorithm(String algorithm) { + this.algorithm = algorithm; + return this; + } + + public Builder setPrivateKeyFile(File privateKeyFile) { + this.privateKeyFile = privateKeyFile; + return this; + } + + public Builder setPassphrase(String passphrase) { + this.passphrase = Optional.of(passphrase); + return this; + } + + private DefaultAssertionCreator build() { + return new DefaultAssertionCreator(algorithm, privateKeyFile, passphrase); + } + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/DynamicAssertionJwtTemplateTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/DynamicAssertionJwtTemplateTest.java new file mode 100644 index 00000000000..54ebc387788 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/DynamicAssertionJwtTemplateTest.java @@ -0,0 +1,83 @@ +/* + * 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.kafka.common.security.oauthbearer.internals.secured.assertion; + +import org.apache.kafka.common.utils.MockTime; + +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashSet; +import java.util.List; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; + +public class DynamicAssertionJwtTemplateTest { + + private final MockTime time = new MockTime(); + + @Test + public void testBasicUsage() throws IOException { + String algorithm = "somealg"; + int expiration = 1; + int notBefore = 20; + boolean includeJti = false; + + try (AssertionJwtTemplate template = new DynamicAssertionJwtTemplate(time, algorithm, expiration, notBefore, includeJti)) { + Map header = template.header(); + assertNotNull(header); + assertEquals("JWT", header.get("typ")); + assertEquals(algorithm, header.get("alg")); + + long currSeconds = time.milliseconds() / 1000L; + + Map payload = template.payload(); + assertNotNull(payload); + assertEquals(currSeconds, payload.get("iat")); + assertEquals(currSeconds + expiration, payload.get("exp")); + assertEquals(currSeconds - notBefore, payload.get("nbf")); + assertNull(payload.get("jti")); + } + } + + @Test + public void testJtiUniqueness() throws IOException { + List jwtIds = new ArrayList<>(); + + for (int i = 0; i < 10; i++) { + try (AssertionJwtTemplate template = new DynamicAssertionJwtTemplate(time, "RSA", 1, 2, true)) { + Map payload = template.payload(); + assertNotNull(payload); + String jwtId = (String) payload.get("jti"); + jwtIds.add(jwtId); + } + } + + // A list of JWT IDs will be the same size as a set if there are no duplicates. + List jwtIds2 = new ArrayList<>(new HashSet<>(jwtIds)); + assertEquals(jwtIds.size(), jwtIds2.size()); + + jwtIds.sort(Comparator.naturalOrder()); + jwtIds2.sort(Comparator.naturalOrder()); + assertEquals(jwtIds, jwtIds2); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/FileAssertionCreatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/FileAssertionCreatorTest.java new file mode 100644 index 00000000000..0dbc6653441 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/FileAssertionCreatorTest.java @@ -0,0 +1,51 @@ +/* + * 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.kafka.common.security.oauthbearer.internals.secured.assertion; + +import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerTest; + +import org.junit.jupiter.api.Test; + +import java.io.File; + +import static org.apache.kafka.test.TestUtils.tempFile; +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class FileAssertionCreatorTest extends OAuthBearerTest { + + @Test + public void testBasicUsage() throws Exception { + String expected = createJwt("jdoe"); + File tmpFile = tempFile(expected); + + try (AssertionCreator assertionCreator = new FileAssertionCreator(tmpFile)) { + String assertion = assertionCreator.create(null); + assertEquals(expected, assertion); + } + } + + @Test + public void testJwtWithWhitespace() throws Exception { + String expected = createJwt("jdoe"); + File tmpFile = tempFile(" " + expected + "\n\n\n"); + + try (AssertionCreator assertionCreator = new FileAssertionCreator(tmpFile)) { + String assertion = assertionCreator.create(null); + assertEquals(expected, assertion); + } + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/FileAssertionJwtTemplateTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/FileAssertionJwtTemplateTest.java new file mode 100644 index 00000000000..edce26946c2 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/assertion/FileAssertionJwtTemplateTest.java @@ -0,0 +1,129 @@ +/* + * 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.kafka.common.security.oauthbearer.internals.secured.assertion; + +import org.apache.kafka.common.KafkaException; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.util.List; +import java.util.Map; + +import static org.apache.kafka.test.TestUtils.tempFile; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class FileAssertionJwtTemplateTest { + + @Test + public void testBasicUsage() throws Exception { + String expected = createTemplateJson( + Map.of("typ", "JWT", "alg", "RS256"), + Map.of("sub", "jdoe") + ); + + File tmpFile = tempFile(expected); + + try (AssertionJwtTemplate template = new FileAssertionJwtTemplate(tmpFile)) { + Map header = template.header(); + assertNotNull(header); + assertEquals("JWT", header.get("typ")); + assertEquals("RS256", header.get("alg")); + + Map payload = template.payload(); + assertNotNull(payload); + assertEquals("jdoe", payload.get("sub")); + } + } + + @Test + public void testHeaderOnly() throws Exception { + String expected = toJson( + Map.of( + "header", + Map.of("typ", "JWT", "alg", "RS256") + ) + ); + + File tmpFile = tempFile(expected); + + try (AssertionJwtTemplate template = new FileAssertionJwtTemplate(tmpFile)) { + Map header = template.header(); + assertNotNull(header); + assertEquals("JWT", header.get("typ")); + assertEquals("RS256", header.get("alg")); + + Map payload = template.payload(); + assertNotNull(payload); + assertTrue(payload.isEmpty()); + } + } + + @Test + public void testPayloadOnly() throws Exception { + String expected = toJson( + Map.of( + "payload", + Map.of("sub", "jdoe") + ) + ); + + File tmpFile = tempFile(expected); + + try (AssertionJwtTemplate template = new FileAssertionJwtTemplate(tmpFile)) { + Map header = template.header(); + assertNotNull(header); + assertTrue(header.isEmpty()); + + Map payload = template.payload(); + assertNotNull(payload); + assertEquals("jdoe", payload.get("sub")); + } + } + + @Test + public void testMalformedFile() throws Exception { + String expected = "{invalid-json}"; + File tmpFile = tempFile(expected); + + assertThrows(KafkaException.class, () -> new FileAssertionJwtTemplate(tmpFile)); + } + + @Test + public void testMalformedFormat() throws Exception { + String expected = toJson(Map.of("header", List.of("foo", "bar", "baz"))); + File tmpFile = tempFile(expected); + + assertThrows(KafkaException.class, () -> new FileAssertionJwtTemplate(tmpFile)); + } + + private String createTemplateJson(Map header, Map payload) { + Map topLevel = Map.of("header", header, "payload", payload); + return toJson(topLevel); + } + + private String toJson(Map map) { + ObjectMapper mapper = new ObjectMapper(); + return assertDoesNotThrow(() -> mapper.writeValueAsString(map)); + } +} diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index fb9b2939f32..77e8449199c 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -946,16 +946,33 @@ class KafkaConfigTest { case SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS => case SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS => case SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS => - case SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME => - case SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME => - case SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL => - case SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL => - case SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS => - case SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS => - case SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS => + case SaslConfigs.SASL_OAUTHBEARER_ASSERTION_ALGORITHM => + case SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_AUD => + case SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_EXP_SECONDS => + case SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_ISS => + case SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_JTI_INCLUDE => + case SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_NBF_SECONDS => + case SaslConfigs.SASL_OAUTHBEARER_ASSERTION_CLAIM_SUB => + case SaslConfigs.SASL_OAUTHBEARER_ASSERTION_FILE => + case SaslConfigs.SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_FILE => + case SaslConfigs.SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_PASSPHRASE => + case SaslConfigs.SASL_OAUTHBEARER_ASSERTION_TEMPLATE_FILE => + case SaslConfigs.SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_ID => + case SaslConfigs.SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_SECRET => case SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS => case SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE => case SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER => + case SaslConfigs.SASL_OAUTHBEARER_HEADER_URLENCODE => + case SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS => + case SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS => + case SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS => + case SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL => + case SaslConfigs.SASL_OAUTHBEARER_JWT_RETRIEVER_CLASS => + case SaslConfigs.SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS => + case SaslConfigs.SASL_OAUTHBEARER_SCOPE => + case SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME => + case SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME => + case SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL => // Security config case SecurityConfig.SECURITY_PROVIDERS_CONFIG => diff --git a/tools/src/main/java/org/apache/kafka/tools/OAuthCompatibilityTool.java b/tools/src/main/java/org/apache/kafka/tools/OAuthCompatibilityTool.java index 7852c3a07e0..40f3100054b 100644 --- a/tools/src/main/java/org/apache/kafka/tools/OAuthCompatibilityTool.java +++ b/tools/src/main/java/org/apache/kafka/tools/OAuthCompatibilityTool.java @@ -24,13 +24,9 @@ import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.config.SaslConfigs; import org.apache.kafka.common.config.SslConfigs; import org.apache.kafka.common.config.types.Password; +import org.apache.kafka.common.security.oauthbearer.JwtRetriever; +import org.apache.kafka.common.security.oauthbearer.JwtValidator; import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule; -import org.apache.kafka.common.security.oauthbearer.internals.secured.CloseableVerificationKeyResolver; -import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtRetriever; -import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtValidator; -import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtRetriever; -import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtValidator; -import org.apache.kafka.common.security.oauthbearer.internals.secured.VerificationKeyResolverFactory; import org.apache.kafka.common.utils.Exit; import net.sourceforge.argparse4j.ArgumentParsers; @@ -44,6 +40,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import javax.security.auth.login.AppConfigurationEntry; + import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS; import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS_DOC; import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS; @@ -116,6 +114,8 @@ import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallb import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_SECRET_DOC; import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.SCOPE_CONFIG; import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.SCOPE_DOC; +import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.ConfigurationUtils.getConfiguredInstance; public class OAuthCompatibilityTool { @@ -133,24 +133,27 @@ public class OAuthCompatibilityTool { ConfigHandler configHandler = new ConfigHandler(namespace); Map configs = configHandler.getConfigs(); - Map jaasConfigs = configHandler.getJaasOptions(); + List jaasConfigEntries = List.of( + new AppConfigurationEntry( + OAuthBearerLoginModule.class.getName(), + AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, + configHandler.getJaasOptions() + ) + ); try { - String accessToken; + String jwt; { // Client side... - try (JwtRetriever atr = new DefaultJwtRetriever(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, jaasConfigs)) { - atr.init(); - - try (JwtValidator atv = new DefaultJwtValidator(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM)) { - atv.init(); + try (JwtRetriever retriever = createRetriever(configs, jaasConfigEntries)) { + try (JwtValidator validator = createValidator(configs, jaasConfigEntries)) { System.out.println("PASSED 1/5: client configuration"); - accessToken = atr.retrieve(); + jwt = retriever.retrieve(); System.out.println("PASSED 2/5: client JWT retrieval"); - atv.validate(accessToken); + validator.validate(jwt); System.out.println("PASSED 3/5: client JWT validation"); } } @@ -158,16 +161,11 @@ public class OAuthCompatibilityTool { { // Broker side... - try (CloseableVerificationKeyResolver vkr = VerificationKeyResolverFactory.create(configs, jaasConfigs)) { - vkr.init(); + try (JwtValidator validator = createValidator(configs, jaasConfigEntries)) { + System.out.println("PASSED 4/5: broker configuration"); - try (JwtValidator atv = new DefaultJwtValidator(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, vkr)) { - atv.init(); - System.out.println("PASSED 4/5: broker configuration"); - - atv.validate(accessToken); - System.out.println("PASSED 5/5: broker JWT validation"); - } + validator.validate(jwt); + System.out.println("PASSED 5/5: broker JWT validation"); } } @@ -186,6 +184,25 @@ public class OAuthCompatibilityTool { } } + private static JwtRetriever createRetriever(Map configs, List jaasConfigEntries) { + return getConfiguredInstance( + configs, + OAUTHBEARER_MECHANISM, + jaasConfigEntries, + SaslConfigs.SASL_OAUTHBEARER_JWT_RETRIEVER_CLASS, + JwtRetriever.class + ); + } + + private static JwtValidator createValidator(Map configs, List jaasConfigEntries) { + return getConfiguredInstance( + configs, + OAUTHBEARER_MECHANISM, + jaasConfigEntries, + SaslConfigs.SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS, + JwtValidator.class + ); + } private static class ArgsHandler {