Status
Current state: "Under Discussion"
Discussion thread: here
JIRA: KAFKA-6562
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
Motivation
The ability to authenticate to Kafka with an OAuth 2 Access Token is desirable given the popularity of OAuth. The introduction of KIP-86 (Configurable SASL callback handlers) makes it possible to non-intrusively add new SASL mechanisms, and OAUTHBEARER is the SASL mechanism for OAuth. OAuth 2 is a flexible framework with multiple ways of doing the same thing (for example, getting a public key to confirm a digital signature), so pluggable implementations – and flexibility in their configuration – is a requirement.
This KIP proposes to add the following functionality related to SASL/OAUTHBEARER:
- Allow clients (both brokers when SASL/OAUTHBEARER is the inter-broker protocol as well as non-broker clients) to flexibly retrieve an access token from an OAuth 2 authorization server according to JAAS-defined configuration (and potentially plugged-in callback handler implementation) and present the access token to a broker for authentication.
- Allow brokers to flexibly validate provided access tokens when a client establishes a connection based on JAAS-defined configuration (and potentially plugged-in callback handler implementation).
- Provide implementations of the above retrieval and validation features based on an unsecured JSON Web Token. This unsecured functionality serves two purposes: first, it provides a way for SASL/OAUTHBEARER to be used in development scenarios out-of-the-box with no OAuth 2 infrastructure required; and second, it provides a way to test the SASL implementation itself. See Rejected Alternatives: Motivation.
- Allow clients (both brokers when SASL/OAUTHBEARER is the inter-broker protocol as well as non-broker clients) to transparently retrieve a new access token in the background before the existing access token expires in case the client has to open new connections. Any existing connections will remain unaffected by this "token refresh" functionality as long as the connection remains intact, but new connections from the same client will always use the latest access token (either the initial one or the one that was most recently retrieved by the token refresh functionality, if any). This is how Kerberos-authenticated connections work with respect to ticket expiration, though this KIP does not attempt to unify the refresh-related code for the two mechanisms.
Note that the access token can be made available to the broker for authorization decisions due to KIP-189 (by exposing the access token via a negotiated property on the SaslServer implementation), but detailed discussion of this possibility is outside the scope of this proposal. It is noted, however, that if access tokens are somehow used for authorization decisions, it is conceivable due to the long-lived nature of Kafka connections that authorization decisions will sometimes be made using expired access tokens. For example, it is up to the broker to validate the token upon authentication, but the token will not be replaced for that particular connection as long as it remains intact; if the token expires in an hour then authorization decisions for that first hour will be made using the still-valid token, but after an hour the expired token would remain associated with the connection, and authorization decisions from that point forward for that particular connection would be made using the expired token. This would have to be addressed via a separate KIP if it turns out to be problematic, but that seems unlikely (code signing certificates that have been timestamped remain valid after their expiration, for example, and access tokens are indeed timestamped). Another issue that would need to be addressed is how to revoke authorizations. Connections are long-lived, and bearer tokens are immutable, so a mechanism to evolve or revoke permissions over time would have to exist. Again, this is outside the scope of this KIP.
Finally, note that the implementation of flexible, substitution-aware configuration that was originally proposed in an early draft of this KIP was deemed more generally useful and has been separated out into its own KIP-269 Substitution Within Configuration Values, which is now a prerequisite for this one.
Public Interfaces
The public interface for this KIP consists of 8 Java classes and 1 Java interface along with various configuration requirements. The following sections define these public-facing parts of this KIP, including an overall UML diagram and important code details (with Javadoc) where appropriate.
Tokens and Token Refresh
See Rejected Alternatives: Token Refresh
We define the org.apache.kafka.common.security.oauthbearer.OAuthBearerToken
interface to be the interface that all OAuth 2 bearer tokens must implement within the context of Kafka's SASL/OAUTHBEARER implementation. Scenarios that leverage open source JWT/JWS/JWE implementations must wrap the library's implementation of a token to implement this interface.
The org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule
class is the JAAS login module that is declared in the JAAS configuration.
The org.apache.kafka.common.security.oauthbearer.OAuthBearerRefreshingLogin
class periodically refreshes the token before it expires so that the client can continue to make connections to brokers, and it is set via the sasl.login.class
client configuration property or the listener.name.sasl_ssl.oauthbearer.sasl.login.class
broker configuration property. It is unlikely that a different Login implementation would be used. The parameters that impact how the refresh algorithm operates are provided as part of the JAAS configuration, and they are as follows:
JAAS Configuration Property | Impact on Refresh Algorithm |
---|
clientRefreshWindowFactor | The background login refresh thread will sleep until the specified window factor relative to the token's total lifetime has been reached, at which time it will try to refresh the credential. Legal values are between 0.5 (50%) and 1.0 (100%) inclusive; a default value of 0.8 (80%) is used if a legal value is not specified. |
clientRefreshWindowJitter | The maximum amount of random jitter relative to the token's total lifetime that is added to the background login refresh thread's sleep time. Legal values are between 0 and 0.25 (25%) inclusive; a default value of 0.05 (5%) is used if a legal value is not specified. |
clientRefreshMinPeriodSeconds | The desired minimum time to wait before refreshing a token, in seconds. Legal values are between 0 and 900 (15 minutes); a default value of 60 (1 minute) is used if a legal value is not specified. |
clientRefreshBufferSeconds | The amount of buffer time before expiration to maintain when refreshing. If a refresh is scheduled to occur closer to expiration than the number of seconds defined here then the refresh will be moved up if possible so as to maintain the desired buffer. Legal values are between 0 and 3,600 (1 hour); a default value of 120 (2 minutes) is used if a legal value is not specified. |
package org.apache.kafka.common.security.oauthbearer;
/**
* The <code>b64token</code> value as defined in
* <a href="https://tools.ietf.org/html/rfc6750#section-2.1">RFC 6750 Section
* 2.1</a> along with the token's specific scope and lifetime and principal
* name.
* <p>
* A network request would be required to re-hydrate an opaque token, and that
* could result in (for example) an {@code IOException}, but retrievers for
* various attributes ({@link #scope()}, {@link #lifetime()}, etc.) declare no
* exceptions. Therefore, if a network request is required for any of these
* retriever methods, that request could be performed at construction time so
* that the various attributes can be reliably provided thereafter. For example,
* a constructor might declare
* {@code throws IOException, OAuthBearerIllegalTokenException} in such a case;
* it might raise {@code IOException} if one or more required networked
* resources (e.g. to re-hydrate an opaque token) is unavailable, and it might
* raise {@code OAuthBearerIllegalTokenException} if there is something
* fundamentally wrong with the token (if it is malformed, for example).
* Alternatively, the retrievers could throw unchecked exceptions.
*
* @see <a href="https://tools.ietf.org/html/rfc6749#section-1.4">RFC 6749
* Section 1.4</a> and
* <a href="https://tools.ietf.org/html/rfc6750#section-2.1">RFC 6750
* Section 2.1</a>
*/
public interface OAuthBearerToken {
/**
* The <code>b64token</code> value as defined in
* <a href="https://tools.ietf.org/html/rfc6750#section-2.1">RFC 6750 Section
* 2.1</a>
*
* @return <code>b64token</code> value as defined in
* <a href="https://tools.ietf.org/html/rfc6750#section-2.1">RFC 6750
* Section 2.1</a>
*/
String value();
/**
* The token's scope of access, as per
* <a href="https://tools.ietf.org/html/rfc6749#section-1.4">RFC 6749 Section
* 1.4</a>
*
* @return the token's (always non-null but potentially empty) scope of access,
* as per <a href="https://tools.ietf.org/html/rfc6749#section-1.4">RFC
* 6749 Section 1.4</a>. Note that all values in the returned set will
* be trimmed of preceding and trailing whitespace, and the result will
* never contain the empty string.
*/
Set<String> scope();
/**
* The token's lifetime, expressed as the number of milliseconds since the
* epoch, as per <a href="https://tools.ietf.org/html/rfc6749#section-1.4">RFC
* 6749 Section 1.4</a>
*
* @return the token'slifetime, expressed as the number of milliseconds since
* the epoch, as per
* <a href="https://tools.ietf.org/html/rfc6749#section-1.4">RFC 6749
* Section 1.4</a>.
*/
long lifetime();
/**
* The name of the principal to which this credential applies
*
* @return the always non-null/non-empty principal name
*/
String principalName();
/**
* When the credential became valid, in terms of the number of milliseconds
* since the epoch, if known, otherwise null. An expiring credential may not
* necessarily indicate when it was created -- just when it expires -- so we
* need to support a null return value here.
*
* @return the time when the credential became valid, in terms of the number of
* milliseconds since the epoch, if known, otherwise null
*/
Long startTimeMillis();
}
package org.apache.kafka.common.security.oauthbearer;
/**
* The {@code LoginModule} for the SASL/OAUTHBEARER mechanism.
*/
public class OAuthBearerLoginModule implements LoginModule {
static {
OAuthBearerSaslClientProvider.initialize(); // not part of public API
OAuthBearerSaslServerProvider.initialize(); // not part of public API
}
// etc...
}
package org.apache.kafka.common.security.oauthbearer;
/**
* This class is responsible for refreshing logins for both Kafka client and
* server when the credential is an OAuth 2 bearer token communicated over
* SASL/OAUTHBEARER. An OAuth 2 bearer token has a limited lifetime, and an
* instance of this class periodically refreshes it so that the client can
* create new connections to brokers on an ongoing basis.
* <p>
* This class is set via the {@code sasl.login.class} client configuration
* property or the {@code listener.name.sasl_ssl.oauthbearer.sasl.login.class}
* broker configuration property.
* <p>
* The login callback handler seen by the {@link OAuthBearerLoginModule}
* instance is set via the {@code sasl.login.callback.handler.class} client
* configuration property or the
* {@code listener.name.sasl_ssl.oauthbearer.sasl.login.callback.handler.class}
* broker configuration property.
* <p>
* This class recognizes the following refresh-related configuration properties,
* which must be set in the JAAS configuration:
* <ul>
* <li><b>clientRefreshWindowFactor</b> -- the background login refresh thread
* will sleep until the specified window factor relative to the token's total
* lifetime has been reached, at which time it will try to refresh the
* credential. Legal values are between 0.5 (50%) and 1.0 (100%) inclusive; a
* default value of 0.8 (80%) is used if a legal value is not specified.</li>
* <li><b>clientRefreshWindowJitter</b> -- the maximum amount of random jitter
* relative to the token's total lifetime that is added to the background login
* refresh thread's sleep time. Legal values are between 0 and 0.25 (25%)
* inclusive; a default value of 0.05 (5%) is used if a legal value is not
* specified.</li>
* <li><b>clientRefreshMinPeriodMillis</b> -- the desired minimum time to wait
* before refreshing a token, in milliseconds. Legal values are between 0 and
* 900,000 (15 minutes); a default value of 60,000 (1 minute) is used if a legal
* value is not specified.
* <li><b>clientRefreshBufferSeconds</b> -- the amount of buffer time before
* expiration to maintain when refreshing. If a refresh is scheduled to occur
* closer to expiration than the number of seconds defined here then the refresh
* will be moved up if possible so as to maintain the desired buffer. Legal
* values are between 0 and 3,600 (1 hour); a default value of 120 (2 minutes)
* is used if a legal value is not specified.</li>
* </ul>
* Note that SASL/OAUTHBEARER logins as managed by this class are only supported
* when a single {@code LoginModule} implementing {@link OAuthBearerLoginModule}
* is communicated to the code. This may be in a non-broker client client where
* only 1 SASL mechanism can be declared; or it may be in an inter-broker
* context where there is only one SASL mechanism defined for the cluster or
* because the JAAS configuration is done via the dynamic functionality
* introduced via <a href=
* "https://cwiki.apache.org/confluence/display/KAFKA/KIP-226+-+Dynamic+Broker+Configuration">KIP-226</a>
* that eliminates the mechanism-to-login-module ambiguity associated with
* declaring multiple SASL mechanisms in a single broker JAAS configuration
* file.
*
* @see OAuthBearerUnsecuredLoginCallbackHandler
*/
public class OAuthBearerRefreshingLogin implements Login {
// etc...
}
Callback Handlers and Callbacks
See Rejected Alternatives: Callback Handlers and Callbacks
We define the abstract base class org.apache.kafka.common.security.oauthbearer.OAuthBearerCallbackHandler
as the base class for all callback handlers related to SASL/OAUTHBEARER. We define the org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallback
class as the callback class for communicating that we want to retrieve a token, and we define the org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback
class as the callback class for communicating that we want to validate a token compact serialization value.
We provide 3 callback handlers. The first is org.apache.kafka.common.security.oauthbearer.OAuthBearerSaslClientCallbackHandler
, and it is responsible for providing the SASL client implementation with the OAuth 2 bearer token that the login module received from the OAuth 2 login event; it is set via the sasl.client.callback.handler.class
configuration property, and it recognizes OAuthBearerLoginCallback
. It is unlikely that an alternative callback handler would be used in place of this one.
The second callback handler defines the token retrieval/authorization server login mechanism that the login module uses to get an OAuth 2 bearer token. This callback handler must be set via the sasl.login.callback.handler.class
client configuration property or the listener.name.sasl_ssl.oauthbearer.sasl.login.callback.handler.class
broker configuration property, and it must recognize OAuthBearerLoginCallback
. The implementation we provide is the org.apache.kafka.common.security.oauthbearer.OAuthBearerUnsecuredLoginCallbackHandler
unsecured JWT implementation. It accepts JAAS options as described in the below Javadoc, and it exists both to provide a way to test the overall SASL/OAUTHBEARER feature set as well as to provide an out-of-the-box implementation for users. An alternative callback handler must be written for production use.
The third callback handler defines the validation mechanism that the SASL Server uses to validate an OAuth 2 bearer token. This callback handler must be set via the listener.name.sasl_ssl.oauthbearer.sasl.server.callback.handler.class
broker configuration property, and it must recognize OAuthBearerValidatorCallback
. The implementation we provide is the org.apache.kafka.common.security.oauthbearer.OAuthBearerUnsecuredValidatorCallbackHandler
unsecured JWT validator. It accepts JAAS options as described in the below Javadoc, and it exists both to provide a way to test the overall SASL/OAUTHBEARER feature set as well as to provide an out-of-the-box implementation for users. An alternative callback handler must be written for production use.
The validated token will be available as a negotiated property on the SASL Server instance with the key OAUTHBEARER.token
so it can be used for authorization as per KIP-189. Note that the implementation of the SASL Server itself is not part of the public interface – just the key where it makes the validated token available.
package org.apache.kafka.common.security.oauthbearer;
/**
* Base class for all SASL/OAUTHBEARER callback handlers. It is a requirement
* that SASL/OAUTHBEARER is the only SASL mechanism configuration presented to
* the code. Specifically, multiple SASL mechanisms configured via the same JAAS
* configuration is not supported; use dynamic configuration via the
* {@code sasl.jaas.config} property as described in <a href-=
* "https://cwiki.apache.org/confluence/display/KAFKA/KIP-226+-+Dynamic+Broker+Configuration">KIP-226</a>
* to meet this constraint if necessary.
*/
public abstract class OAuthBearerCallbackHandler implements AuthenticateCallbackHandler {
private Map<String, ?> serverConfig = null;
private String mechanism = null;
private AppConfigurationEntry jaasConfigEntry = null;
private SubstitutableValues substitutableValues = null;
/**
* Return the server configuration provided during
* {@link #configure(Map, String, List)}, if any, otherwise null
*
* @return the server configuration provided during
* {@link #configure(Map, String, List)}, if any, otherwise null
*/
public Map<String, ?> serverConfig() {
return serverConfig;
}
/**
* Return the SASL mechanism provided during
* {@link #configure(Map, String, List)}, if any, otherwise null
*
* @return the SASL mechanism provided during
* {@link #configure(Map, String, List)}, if any, otherwise null
*/
public String mechanism() {
return mechanism;
}
/**
* Return the JAAS login module configuration provided during
* {@link #configure(Map, String, List)}, if any, otherwise null.
*
* @return the JAAS login module configuration provided during
* {@link #configure(Map, String, List)}, if any, otherwise null
*/
public AppConfigurationEntry jaasConfigEntry() {
return jaasConfigEntry;
}
/**
* Return the substitutableValues as determined via
* {@link #configure(Map, String, List)}
*
* @return the substitutableValues as determined via
* {@link #configure(Map, String, List)}
*/
public SubstitutableValues substitutableValues() {
return substitutableValues;
}
// etc...
}
package org.apache.kafka.common.security.oauthbearer;
/**
* A {@code Callback} for use by the {@code SaslClient} and {@code Login}
* implementations when they require an OAuth 2 bearer token. Callback handlers
* should use the {@link #error(String, String, String)} method to communicate
* errors returned by the authorization server as per
* <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
* 2.0 Authorization Framework</a>. Callback handlers should communicate other
* problems by raising an {@code IOException}.
*/
public class OAuthBearerLoginCallback implements Callback {
private OAuthBearerToken token = null;
private String errorCode = null;
private String errorDescription = null;
private String errorUri = null;
/**
* Return the (potentially null) token
*
* @return the (potentially null) token
*/
public OAuthBearerToken token() {
return token;
}
/**
* Return the (always non-empty) error code as per
* <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
* 2.0 Authorization Framework</a>.
*
* @return the (always non-empty) error code
*/
public String errorCode() {
return errorCode;
}
/**
* Return the (potentially null) error description as per
* <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
* 2.0 Authorization Framework</a>.
*
* @return the (potentially null) error description
*/
public String errorDescription() {
return errorDescription;
}
/**
* Return the (potentially null) error URI as per
* <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
* 2.0 Authorization Framework</a>.
*
* @return the (potentially null) error URI
*/
public String errorUri() {
return errorUri;
}
/**
* Set the token. All error-related values are cleared.
*
* @param token
* the mandatory token to set
*/
public void token(OAuthBearerToken token) {
this.token = Objects.requireNonNull(token);
this.errorCode = null;
this.errorDescription = null;
this.errorUri = null;
}
/**
* Set the error values as per
* <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
* 2.0 Authorization Framework</a>. Any token is cleared.
*
* @param errorCode
* the mandatory error code to set
* @param errorDescription
* the optional error description to set
* @param errorCode
* the optional error URI to set
*/
public void error(String errorCode, String errorDescription, String errorUri) {
if (Objects.requireNonNull(errorCode).isEmpty())
throw new IllegalArgumentException("error code must not be empty");
this.errorCode = errorCode;
this.errorDescription = errorDescription;
this.errorUri = errorUri;
this.token = null;
}
}
package org.apache.kafka.common.security.oauthbearer;
/**
* A {@code Callback} for use by the {@code SaslServer} implementation when it
* needs to provide an OAuth 2 bearer token compact serialization for
* validation. Callback handlers should use the
* {@link #error(String, String, String)} method to communicate errors back to
* the SASL Client as per
* <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
* 2.0 Authorization Framework</a> and the <a href=
* "https://www.iana.org/assignments/oauth-parameters/oauth-parameters.xhtml#extensions-error">IANA
* OAuth Extensions Error Registry</a>. Callback handlers should communicate
* other problems by raising an {@code IOException}.
*/
public class OAuthBearerValidatorCallback implements Callback {
private final String tokenValue;
private OAuthBearerToken token = null;
private String errorStatus = null;
private String errorScope = null;
private String errorOpenIDConfiguration = null;
/**
* Constructor
*
* @param tokenValue
* the mandatory/non-blank token value
*/
public OAuthBearerValidatorCallback(String tokenValue) {
if (Objects.requireNonNull(tokenValue).isEmpty())
throw new IllegalArgumentException("token value must not be empty");
this.tokenValue = tokenValue;
}
/**
* Return the (always non-null) token value
*
* @return the (always non-null) token value
*/
public String tokenValue() {
return tokenValue;
}
/**
* Return the (potentially null) token
*
* @return the (potentially null) token
*/
public OAuthBearerToken token() {
return token;
}
/**
* Return the (potentially null) error status value as per
* <a href="https://tools.ietf.org/html/rfc7628#section-3.2.2">RFC 7628: A Set
* of Simple Authentication and Security Layer (SASL) Mechanisms for OAuth</a>
* and the <a href=
* "https://www.iana.org/assignments/oauth-parameters/oauth-parameters.xhtml#extensions-error">IANA
* OAuth Extensions Error Registry</a>.
*
* @return the (potentially null) error status value
*/
public String errorStatus() {
return errorStatus;
}
/**
* Return the (potentially null) error scope value as per
* <a href="https://tools.ietf.org/html/rfc7628#section-3.2.2">RFC 7628: A Set
* of Simple Authentication and Security Layer (SASL) Mechanisms for OAuth</a>.
*
* @return the (potentially null) error scope value
*/
public String errorScope() {
return errorScope;
}
/**
* Return the (potentially null) error openid-configuration value as per
* <a href="https://tools.ietf.org/html/rfc7628#section-3.2.2">RFC 7628: A Set
* of Simple Authentication and Security Layer (SASL) Mechanisms for OAuth</a>.
*
* @return the (potentially null) error openid-configuration value
*/
public String errorOpenIDConfiguration() {
return errorOpenIDConfiguration;
}
/**
* Set the token. The token value is unchanged and is expected to match the
* provided token's value. All error values are cleared.
*
* @param token
* the mandatory token to set
*/
public void token(OAuthBearerToken token) {
this.token = Objects.requireNonNull(token);
this.errorStatus = null;
this.errorScope = null;
this.errorOpenIDConfiguration = null;
}
/**
* Set the error values as per
* <a href="https://tools.ietf.org/html/rfc7628#section-3.2.2">RFC 7628: A Set
* of Simple Authentication and Security Layer (SASL) Mechanisms for OAuth</a>.
* Any token is cleared.
*
* @param errorStatus
* the mandatory error status value from the <a href=
* "https://www.iana.org/assignments/oauth-parameters/oauth-parameters.xhtml#extensions-error">IANA
* OAuth Extensions Error Registry</a> to set
* @param errorScope
* the optional error scope value to set
* @param errorStatus
* the optional error openid-configuration value to set
*/
public void error(String errorStatus, String errorScope, String errorOpenIDConfiguration) {
if (Objects.requireNonNull(errorStatus).isEmpty())
throw new IllegalArgumentException("error status must not be empty");
this.errorStatus = errorStatus;
this.errorScope = errorScope;
this.errorOpenIDConfiguration = errorOpenIDConfiguration;
this.token = null;
}
}
package org.apache.kafka.common.security.oauthbearer;
/**
* A {@code CallbackHandler} that recognizes {@link OAuthBearerLoginCallback}
* and provides the OAuth 2 bearer token that was created when the
* {@code OAuthBearerLoginModule} logged in.
* <p>
* This class is set via the {@code sasl.client.callback.handler.class} client
* configuration property.
*/
public class OAuthBearerSaslClientCallbackHandler extends OAuthBearerCallbackHandler {
// etc...
}
package org.apache.kafka.common.security.oauthbearer;
/**
* A {@code CallbackHandler} that recognizes {@link OAuthBearerLoginCallback}
* and returns an unsecured OAuth 2 bearer token.
* <p>
* Claims and their values on the returned token can be specified using
* {@code unsecuredLoginStringClaim_<claimname>},
* {@code unsecuredLoginNumberClaim_<claimname>}, and
* {@code unsecuredLoginListClaim_<claimname>} options. The first character of
* the value is taken as the delimiter for list claims. You may define any claim
* name and value except '{@code iat}' and '{@code exp}', both of which are
* calculated automatically.
* <p>
* This implementation also accepts the following options:
* <ul>
* <li>{@code unsecuredLoginPrincipalClaimName} set to a custom claim name if
* you wish the name of the String claim holding the principal name to be
* something other than '{@code sub}'.</li>
* <li>{@code unsecuredLoginLifetimeSeconds} set to an integer value if the
* token expiration is to be set to something other than the default value of
* 3600 seconds (which is 1 hour). The '{@code exp}' claim reflects the
* expiration time.</li>
* <li>{@code unsecuredLoginScopeClaimName} set to a custom claim name if you
* wish the name of the String or String List claim holding any token scope to
* be something other than '{@code scope}'</li>
* </ul>
* For example:
*
* <pre>
* KafkaClient {
* org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule Required
* unsecuredLoginStringClaim_sub="thePrincipalName"
* unsecuredLoginListClaim_scope="|scopeValue1|scopeValue2"
* unsecuredLoginLifetimeSeconds="60";
* };
* </pre>
*
* This class is set via the {@code sasl.login.callback.handler.class} client
* configuration property.
*/
public class OAuthBearerUnsecuredLoginCallbackHandler extends OAuthBearerCallbackHandler {
// etc...
}
package org.apache.kafka.common.security.oauthbearer;
/**
* A {@code CallbackHandler} that recognizes
* {@link OAuthBearerValidatorCallback} and validates an unsecured OAuth 2
* bearer token. It requires there to be an <code>"exp" (Expiration Time)</code>
* claim of type Number. If <code>"iat" (Issued At)</code> or
* <code>"nbf" (Not Before)</code> claims are present each must be a number that
* precedes the Expiration Time claim, and if both are present the Not Before
* claim must not precede the Issued At claim. It also accepts the following
* options, none of which are required:
* <ul>
* <li>{@code unsecuredValidatorPrincipalClaimName} set to a non-empty value if
* you wish a String claim holding a principal name to be checked for existence;
* the default is to perform no such check</li>
* <li>{@code unsecuredValidatorScopeClaimName} set to a custom claim name if
* you wish the name of the String or String List claim holding any token scope
* to be something other than '{@code scope}'</li>
* <li>{@code unsecuredValidatorRequiredScope} set to a space-delimited list of
* scope values if you wish the String/String List claim holding the token scope
* to be checked to make sure it contains certain values</li>
* <li>{@code unsecuredValidatorAllowableClockSkewMillis} set to a positive
* integer value if you wish to allow up to some number of positive milliseconds
* of clock skew (the default is 0)</li>
* <ul>
* For example:
*
* <pre>
* KafkaServer {
* org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule Required
* unsecuredLoginStringClaim_sub="thePrincipalName"
* unsecuredLoginListClaim_scope=",KAFKA_BROKER,LOGIN_TO_KAFKA"
* unsecuredValidatorPrincipalClaimName="sub"
* unsecuredValidatorRequiredScope="LOGIN_TO_KAFKA"
* unsecuredValidatorAllowableClockSkewMillis="3000";
* };
* </pre>
*
* This class is set via the
* {@code listener.name.sasl_ssl.oauthbearer.sasl.server.callback.handler.class}
* broker configuration property.
*/
public class OAuthBearerUnsecuredValidatorCallbackHandler extends OAuthBearerCallbackHandler {
// etc...
}
Summary of Configuration Requirements
The following table summarizes the proposed configuration requirements for Kafka's SASL/OAUTHBEARER implementation. Note that many configuration options are available only via KIP-86, which is a dependency for this KIP, and one of them is not yet implemented as part of KIP-86 but is required.
Configuration | Example Value | Likelihood of Value Being Different from the Example Value | Notes |
---|
Non-Broker: sasl.login.class
Broker: listener.name.sasl_ssl.oauthbearer.
sasl.login.class
| org.apache.kafka.common.security.oauthbearer.
OAuthBearerRefreshingLogin
| Low | Configiration property only available via KIP-86 |
JAAS Login Module | org.apache.kafka.common.security.oauthbearer.
OAuthBearerLoginModule
| Low | |
sasl.client.callback.handler.class | org.apache.kafka.common.security.oauthbearer. OAuthBearerSaslClientCallbackHandler
| Low | Configiration property only available via KIP-86 |
Non-Broker: sasl.login.callback.handler.class
Broker: listener.name.sasl_ssl.oauthbearer.
sasl.login.callback.handler.class
| org.apache.kafka.common.security.oauthbearer.
OAuthBearerUnsecuredLoginCallbackHandler
| High | Configiration property only available via KIP-86 (not yet formally part of that KIP) |
listener.name.sasl_ssl.oauthbearer.
sasl.server.callback.handler.class
| org.apache.kafka.common.security.oauthbearer.
OAuthBearerUnsecuredValidatorCallbackHandler
| High | Configiration property only available via KIP-86 |
JAAS Module Options | Varied | High | Used to adjust the token refresh algorithm and to configure the sasl.client and sasl.server callback handlers. |
Proposed Changes
Thanks to KIP-86, no changes to the existing code base are required – all functionality represents additions (rather than changes) to the existing code base.
Compatibility, Deprecation, and Migration Plan
- What impact (if any) will there be on existing users?
None
- If we are changing behavior how will we phase out the older behavior?
There is no change to existing behavior
Rejected Alternatives
Motivation
The idea of providing a toolkit of reusable JWT/JWS/JWE retrieval and validation functionality was discarded in favor of a single, simple unsecured JWS implementation. Anything beyond simple unsecured use cases requires significant functionality that is available via multiple open source libraries, and there is no need to duplicate that functionality here. It is also not desirable for the Kafka project to define a specific open source JWT/JWS/JWE library dependency; better to allow installations to use the library that they feel most comfortable with. This decision also allows the public-facing interface of this KIP to be considerably smaller than it would otherwise be.
Callback Handlers and Callbacks
We decided to leverage the standard JAAS Callback/CallbackHandler mechanism for communicating information between various components (specifically, between the SASL Client and the Login Module, between the Login Module and the Login/Token Retrieval mechanism, and between the SASL Server and the Token Validation mechanism). We had originally documented the need for just the last two (token retrieval and token validation), and the original proposal was to declare the classes as part of the JAAS configuration. The only real benefit to doing this was that the declaration of the classes and their configuration were co-located in the JAAS configuration. We decided the benefit of consistency was at least as valuable as any cost associated with separating the decalartion from the configuration – and probably more valuable given that this is how configuration is done for other SASL mechanisms and is therefore not unfamiliar. We also now leverrage callback handlers in all three places where it is supported.
Token Refresh
We explicitly decide not to unify the refresh logic required for both SASL/OAUTHBEARER and SASL/GSSAPI mechanisms as part of this KIP. This unification could occur at some point in the future. The org.apache.kafka.common.security.oauthbearer.OAuthBearerRefreshingLogin
class delegates to an underlying imlementation in the org.apache.kafka.common.security.oauthbearer.internal
package (this package is explicitly NOT part of the public API), and this delegation-based approach points to a possible way forward with regard to unification, but unification is explicitly out of scope here. This decision allows several classes to be moved to the internal package, and that helps to minimize the public-facing API for this KIP.