Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.
Comment: Adopted in 2.2

Table of Contents

Status

Current stateUnder DiscussionAdopted (in 2.2)

Discussion thread: here

JIRA: KAFKA-7352

...

The adoption of KIP-255: OAuth Authentication via SASL/OAUTHBEARER in release 2.0.0 creates the possibility of using information in the bearer token to make authorization decisions.  Unfortunately, however, Kafka connections are long-lived, so there is no ability to change the bearer token associated with a particular connection.  Allowing SASL connections to periodically re-authenticate would resolve this.  In addition to this motivation there are two others that are security-related.  First, to eliminate access to Kafka for connected clients, the current requirement is to remove all authorizations (i.e. remove all ACLs).  This is necessary because of the long-lived nature of the connections.  It is operationally simpler to shut off access at the point of authentication, and with the release of KIP-86: Configurable SASL Callback Handlers it is going to become more and more likely that installations will authenticate users against external directories (e.g. via LDAP).  The ability to stop Kafka access by simply disabling an account in an LDAP directory (for example) is desirable.  The second motivating factor for re-authentication related to security is that the use of short-lived tokens is a common OAuth security recommendation, but issuing a short-lived token to a Kafka client (or a broker when OAUTHBEARER is the inter-broker protocol) currently has no benefit because once a client is connected to a broker the client is never challenged again and the connection may remain intact beyond the token expiration time (and may remain intact indefinitely under perfect circumstances).  This KIP proposes adding the ability for SASL /OAUTHBEARER clients (and brokers when OAUTHBEARER a SASL mechanism is the inter-broker protocol) to re-authenticate their connections to brokers and have the new bearer token appear on their session rather than the old one.  This KIP also proposes to add the ability for for brokers to close connections that continue to use expired credentials.sessions.

This KIP has no impact on non-SASL connections The implementation is designed in such a way that it does not preclude adding support for re-authentication of other SASL mechanism (e.g. PLAIN, SCRAM-related, and GSSAPI), but doing so is explicitly out-of-scope for this proposal (a section below discusses how this could be done)connections that use the PLAINTEXT or SSL security protocols) – no such connection will be re-authenticated, and no such connection will be closed.

Public Interfaces

This KIP proposes the addition of a single interface to the API and two additional configuration options configuration option to enable the client-side re-authentication and server-side expired-connection-kill features feature (both the option defaults result default results in no functionality change, of course, so there is no change to existing behavior in the absence of an explicit opt-insin).  This KIP also proposes bumping the version number for the SASL_HANDSHAKE API and adjusting the AUTHENTICATE API to 1 (with a change in wire format) so that brokers can indicate the session expiration time to clients via an additional value on the last round-trip response.  Clients can use the max SASL_AUTHENTICATE version number supported by the server to determine if the broker they are connected to a broker that supports re-authentication and they can learn what maximum credential lifetime the broker supports (see below for details(true if version > 0).  It This KIP also adds new metrics as described below.

The interface configuration option this KIP proposes to add to enable server-side expired-connection-kill is as follows:

Code Block
languagejava
titleorg.apache.kafka.common.security.expiring.ExpiringCredential
collapsetrue
/**
 * A credential that expires and that can potentially be refreshed; such a
 * refreshed credential can also potentially be used to re-authenticate an
 * existing connection.
 * <p>
 * The parameters that impact how the refresh algorithm operates are specified
 * as part of the producer/consumer/broker configuration and are as follows. See
 * the documentation for these properties elsewhere for details.
 * <table>
 * <tr>
 * <th>Producer/Consumer/Broker Configuration Property</th>
 * </tr>
 * <tr>
 * <td>{@code sasl.login.refresh.window.factor}</td>
 * </tr>
 * <tr>
 * <td>{@code sasl.login.refresh.window.jitter}</td>
 * </tr>
 * <tr>
 * <td>{@code sasl.login.refresh.min.period.seconds}</td>
 * </tr>
 * <tr>
 * <td>{@code sasl.login.refresh.min.buffer.seconds}</td>
 * </tr>
 * <tr>
 * <td>{@code sasl.login.refresh.reauthenticate.enable}</td>
 * </tr>
 * </table>
 * <p>
 * This interface was introduced in 2.1.0 and, while it feels stable, it could
 * evolve. We will try to evolve the API in a compatible manner, but we reserve
 * the right to make breaking changes in minor releases, if necessary. We will
 * update the {@code InterfaceStability} annotation and this notice once the API
 * is considered stable.
 * 
 * @see SaslConfigs#SASL_LOGIN_REFRESH_WINDOW_FACTOR_DOC
 * @see SaslConfigs#SASL_LOGIN_REFRESH_WINDOW_JITTER_DOC
 * @see SaslConfigs#SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_DOC
 * @see SaslConfigs#SASL_LOGIN_REFRESH_BUFFER_SECONDS_DOC
 * @see SaslConfigs#SASL_LOGIN_REFRESH_REAUTHENTICATE_ENABLE_DOC
 */
@InterfaceStability.Evolving
public interface ExpiringCredential {
    /**
     * The value to use when asking a {@code SaslClient} instance for the
     * {@code ExpiringCredential} that was used to authenticate, if any
     */
    public static final String SASL_CLIENT_NEGOTIATED_PROPERTY_NAME = "Expiring.Credential";

    /**
     * The name of the principal to which this credential applies (used only for
     * logging)
     * 
     * @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 startTimeMs();

    /**
     * When the credential expires, in terms of the number of milliseconds since the
     * epoch. All expiring credentials by definition must indicate their expiration
     * time -- thus, unlike other methods, we do not support a null return value
     * here.
     * 
     * @return the time when the credential expires, in terms of the number of
     *         milliseconds since the epoch
     */
    long expireTimeMs();

    /**
     * The point after which the credential can no longer be refreshed, in terms of
     * the number of milliseconds since the epoch, if any, otherwise null. Some
     * expiring credentials can be refreshed over and over again without limit, so
     * we support a null return value here.
     * 
     * @return the point after which the credential can no longer be refreshed, in
     *         terms of the number of milliseconds since the epoch, if any,
     *         otherwise null
     */
    Long absoluteLastRefreshTimeMs();
}

The configuration option this KIP proposes to add to enable client-side re-authentication is 'sasl.login.refresh.reauthenticate.enable' – it defaults to false, and when explicitly set to true in the context of a client (including a broker when it acts as an inter-broker client) the client-side re-authentication feature will be enabled for any SASL connection that uses a private credential implementing the above ExpiringCredential interface (this will be the case with OAUTHBEARER by default because the Login implementation that is used by default for OAUTHBEARER will be amended as part of this KIP to automatically wrap any OAuthBearerToken instance to make it implement ExpiringCredential if it does not already do so).

As mentioned above, the SASL_HANDSHAKE API will have its version number bumped (and its wire format changed as described below) so that any client with the above configuration value set to true will not try to re-authenticate to a broker that has not been upgraded to the necessary version and therefore does not support responding to such requests.

The configuration option this KIP proposes to add to enable server-side expired-connection-kill is 'connections.max.reauth.ms' – it must be prefixed with listener prefix and SASL mechanism name in lower-case. For example, "sasl_ssl.oauthbearer.connections.max.reauth.ms=3600000".  The value defaults to 0.  When explicitly set to a non-zero value the server will reject any authentication or re-authentication attempt from a client that presents a bearer token whose lifetime exceeds the time at which the (re-)authentication occurs plus a number of milliseconds equal to the absolute value of the configured value (for example, the token lifetime must not exceed one hour past the time of (re-)authentication if the configured value is either -3600000 or +3600000).  When explicitly set to a positive number, in addition to the lifetime check, the server will disconnect any connection that does not re-authenticate and subsequently uses the connection for any purpose other than re-authentication at any point beyond the expiration point.  For example, if the configured value is 3600000 and the remaining token lifetime at the time of authentication is 45 minutes, the server would kill the connection if it is not re-authenticated within 45 minutes and it is then actively used for anything other than re-authentication.  As a further example, if the configured value is 3600000 and the mechanism is not OAUTHBEARER (e.g. it is PLAIN, SCRAM-related, or GSSAPI) then the server would kill the connection as soon as it is used more than 1 hour after the time of (re-)authentication (assuming the mechanism does not support re-authentication, which it will not for these mechanisms when this KIP is first delivered – but it could be added at a later time as described below in "How To Support Re-Authentication for Other SASL Mechanisms").

The 'connections.max.reauth.ms' configuration option supports positive and negative values to facilitate migration; typically the value will first be set to a negative value and then it will be converted to its absolute value to fully enable the feature once metrics indicate all clients are upgraded and re-authenticating (see Migration Plan for details).

Neither of the above configuration options will be dynamically changeable, so restarts will be required if either value is to be changed.

From a behavior perspective on the client side (again, including the broker when it is acting as an inter-broker client), when the client-side re-authentication option is enabled and a private credential implements ExpiringCredential, the existing configuration options sasl.login.refresh.{window.factor, window.jitter, min.period.seconds, min.buffer.seconds} define when the org.apache.kafka.common.security.auth.Login implementation on the client must generate a new private credential (e.g. when it must retrieve a new bearer token).  The default configuration values result in a new credential being retrieved sometime between 75% and 85% of the current credential's lifetime (e.g. sometime between 45 and 51 minutes after a token with an hour lifetime is initially retrieved).  At that time, when the new credential is retrieved, every connection that authenticated with the old credential and is connected to a broker that supports authentication will be told to re-authenticate.  Re-authentication will begin as soon as the connection is used.  If the re-authentication attempt fails then the connection will be closed by the broker; retries are not supported.  If re-authentication succeeds then any requests that queued up during re-authentication will subsequently be able to flow through, and eventually the connection will be told to re-authenticate again, etc.

From a behavior perspective on the server (broker) side, when the broker-side expired-connection-kill feature is fully enabled with a positive value the broker will close a connection authenticated via the indicated SASL mechanism when the connection is used past the expiration time and the specific API request is not directly related to re-authentication (ApiVersionsRequest, SaslHandshakeRequest, and SaslAuthenticateRequest).  In other words, if a connection sits idle, it will not be closed – something unrelated to re-authentication must traverse the connection before a disconnect will occur.

Metrics documenting re-authentications will be maintained.  They will mirror existing metrics that document authentications.  For example: failed-reauthentication-{rate,total} and successful-reauthentication-{rate,total}.

A broker metric will be created that documents the number of API requests unrelated to re-authentication that are made over a connection that is considered expired.  This metric may be non-zero only when the configuration value is negative.  It helps operators ensure that all clients are properly upgraded and re-authenticating before fully turning on server-side expired-connection-kill functionality (by changing the negative configuration value to its absolute value): the metric will be unchanging across all brokers when it is safe to fully enable the feature with the absolute value.

A broker metric will be created that documents the number connections killed by the server-side expired-connection-kill functionality.  This metric may be non-zero only when the configuration value is positive, and it indicates that a client is connecting to the broker with re-authentication either unavailable (i.e. an older client) or disabled.

A client metric will be created that documents the latency imposed by re-authentication.  It is unclear if this latency will be problematic, and the data collected via this metric may be useful as we consider this issue in the future.

Note that the class that implements the refresh logic on the client side (org.apache.kafka.common.security.expiring.internals.ExpiringCredentialRefreshingLogin) is not considered part of the public API.  This means that while it is up to the org.apache.kafka.common.security.auth.Login implementation for a particular mechanism to implement the logic, and that implementation can delegate to ExpiringCredentialRefreshingLogin to do so (as org.apache.kafka.common.security.oauthbearer.internals.OAuthBearerRefreshingLogin does), this is something that could only be be done in a supported way for the built-in SASL mechanisms (e.g. PLAIN, SCRAM-related, and GSSAPI).  There is no intent to formally support an ability for non-builtin mechanisms to generate credentials that can be refreshed and re-authenticated – nothing would prevent anyone from implementing such mechanisms and delegating to the class, of course, but it would be an unsupported use of an internal class that is not part of the public API.

Proposed Changes

Implementation Overview

The description of this KIP is actually quite straightforward from a behavior perspective – turn the feature on with the configuration options in both the client and the broker and it just works for OAUTHBEARER; use a custom LoginModule for other mechanisms to create credentials implementing ExpiringCredential and delegate to org.apache.kafka.common.security.expiring.internals.ExpiringCredentialRefreshingLogin and it can be made to work on the client for those mechanisms, too (see "How To Support Re-Authentication for Other SASL Mechanisms" for details).  From an implementation perspective, though, the KIP is not so straightforward; a description of how it works therefore follows below.  Note that this description applies to the implementation only – none of this is part of the public API.

This implementation works at a very low level in the Kafka stack, at the level of the network code.  It is therefore transparent to all clients – it just works with no knowledge or accommodation required on their part.  When a client makes a request to a broker the request is intercepted at the level of the Selector class and a check is done to see if re-authentication is enabled; if it is, and the broker supports re-authentication, then the connection is re-authenticated at that point before the request is allowed to flow through.  The solution is elegant because it re-uses existing code paths.

This KIP transparently adds re-authentication support for all uses, which at this point includes the following:

  • org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient
    • org.apache.kafka.clients.consumer.KafkaConsumer
    • org.apache.kafka.connect.runtime.distributed.WorkerGroupMember
  • kafka.controller.ControllerChannelManager
  • org.apache.kafka.clients.admin.KafkaAdminClient
  • org.apache.kafka.clients.producer.KafkaProducer
  • kafka.coordinator.transaction.TransactionMarkerChannelManager
  • kafka.server.ReplicaFetcherBlockingSend (kafka.server.ReplicaFetcherThread)
  • kafka.admin.AdminClient
  • kafka.tools.ReplicaVerificationTool
  • kafka.server.KafkaServer

  • org.apache.kafka.trogdor.workload.ConnectionStressWorker

The final issue to describe is how/when a KafkaChannel instance (each of which corresponds to a unique network connection) is told to re-authenticate.

We define the class org.apache.kafka.common.security.expiring.internals.ClientChannelCredentialTracker to keep track of the various KafkaChannel instances and the ExpiringCredential instances that they authenticated with.  The following code in SaslChannelBuilder adds an instance of ClientChannelCredentialTracker to the private credentials of the Subject associated with the SASL mechanism when the feature is enabled:

Code Block
languagejava
titleSaslChannelBuilder Code to Enable the Feature
LoginManager loginManager = LoginManager.acquireLoginManager(entry.getValue(), mechanism, defaultLoginClass, configs);
loginManagers.put(mechanism, loginManager);
Subject subject = loginManager.subject();
if (mode == Mode.CLIENT) {
    if (saslLoginRefreshReauthenticateEnable()) {
        log.info("SASL Login Refresh Re-Authenticate ENABLED");
        if (subject.getPrivateCredentials(ClientChannelCredentialTracker.class).isEmpty())
            subject.getPrivateCredentials().add(new ClientChannelCredentialTracker());
    } else
        log.info("SASL Login Refresh Re-Authenticate DISABLED");
}

The KafkaChannel and ExpiringCredentialRefreshingLogin classes can easily retrieve the ClientChannelCredentialTracker instance and tell it when various events of interest occur (a channel is initially authenticated, a credential is refreshed, a channel successfully re-authenticates, a channel is disconnected).  The events are enqueued as they are reported, and a background thread processes them to decide which KafkaChannel instances need to re-authenticate.  There is a method KafkaChannel.readyForReauthentication() that marks the instance as requiring re-authentication so the Selector will know.

...

'connections.max.reauth.ms' (not required to be prefixed with listener prefix or SASL mechanism name – such a value would be used across the cluster – but it may be as mentioned below). For example, "connections.max.reauth.ms=3600000".  The value represents the maximum value that could potentially be communicated as part of the new V1 SaslAuthenticateResponse.  The default value is 0, which means there is effectively no maximum communicated (0 will be sent, meaning "none"), server-side kill of expired connections is disabled, clients are not required to re-authenticate, and whether clients re-authenticate or not and at what interval is entirely up to them.  Existing SASL clients upgraded to v2.2.0 will be coded to not re-authenticate in this scenario.  The default value of 0 therefore results in no change whatsoever.

When 'connections.max.reauth.ms' is explicitly set to a positive number the server will disconnect any SASL connection that does not re-authenticate and subsequently uses the connection for any purpose other than re-authentication at any point beyond the communicated expiration point (which will not exceed the configured maximum value).  For example, if the configured value is 3600000 (1 hour) and the remaining lifetime of a bearer token presented at the time of authentication is 45 minutes, then 45 minutes is communicated back to the client and the server would kill the connection if it is not re-authenticated within 45 minutes and it is then actively used for anything other than re-authentication.  As a further example, if the configured value is 3600000 and the credential lifetime is either unspecified or greater than 1 hour then 1 hour would be communicated to the client and the server would kill the connection if it is not re-authenticated within 1 hour and it is then actively used for anything other than re-authentication.

Older clients will of course not have the session expiration time communicated to them since they will use a version 0 SaslAuthenticateRequest and will receive the existing version 0 SaslAuthenticateResponse.  The broker will disconnect these connections upon session expiration regardless of the fact that the client is an older one.  Such connections will be captured via a metric (described below) to help with migration.

The 'connections.max.reauth.ms' configuration option will not be dynamically changeable; restarts will be required if the value is to be changed.  However, if a new listener is dynamically added, the value could be set for that listener at that time (and the configuration key would be prefixed in that case); this dynamic capability will be addressed as a separate ticket and may not be delivered with the initial KIP implementation.

From a behavior perspective on the client side (including the broker when it is acting as an inter-broker client), when a v2.2.0-or-later SASL client connects to a v2.2.0 or later broker that supports re-authentication, the broker will communicate the session expiration time as part of the final SASL_AUTHENTICATE response.  If this value is positive, then the client will automatically re-authenticate before anything else unrelated to re-authentication is sent beyond that expiration point.  If the re-authentication attempt fails then the connection will be closed by the broker; retries are not supported.  If re-authentication succeeds then any received responses that queued up during re-authentication along with the Send that triggered the re-authentication to occur in the first place will subsequently be able to flow through (back to the client and along to the broker, respectively), and eventually the connection will re-authenticate again, etc.  Note also that the client cannot queue up additional send requests beyond the one that triggers re-authentication to occur until re-authentication succeeds and the triggering one is sent.

From a behavior perspective on the server (broker) side, when the expired-connection-kill feature is enabled with a positive value the broker will communicate a session time via SASL_AUTHENTICATE and will close a connection when the connection is used past the expiration time and the specific API request is not directly related to re-authentication (SaslHandshakeRequest and SaslAuthenticateRequest).  In other words, if a connection sits idle, it will not be closed – something unrelated to re-authentication must traverse the connection before a disconnect will occur.

Metrics documenting re-authentications will be maintained in the Selector code.  SOme will mirror existing metrics that document authentications.  For example: failed-reauthentication-{rate,total} and successful-reauthentication-{rate,total}.  There will also be separate successful-authentication-no-reauth-{rate,total} metrics to indicate the subset of clients that successfully authenticate with a V0 SaslAuthenticateRequest (or no such request, which can happen with very old clients)   These metrics are helpful during migration (see Migration Plan for details) as they will identify if/when all clients are properly upgraded before server-side expired-connection-kill functionality is enabled: the rate metric will be zero across all brokers when it is appropriate to enable the feature, and the total metric will be unchanging (zero after a restart).  There will also be reauthentication-latency-{avg,max} metrics that document the latency imposed by re-authentication.  It is unclear if this latency will be problematic, and the data collected via these metrics may be useful as we consider this issue in the future.

An additional metric ExpiredConnectionsKilledCount will be created and maintained by the server-side Processor code to count the number of such events.  It should remain at zero if all clients and brokers are upgraded to v2.2.0 or later.  If it is non-zero then either an older client is connecting (which would be evidenced in the successful-authentication-no-reauth metrics mentioned above) or a newer client is not re-authenticating correctly (which would indicate a bug).

Proposed Changes

Implementation Overview

The description of this KIP is actually quite straightforward from a behavior perspective – turn the feature on with the configuration option in the broker and it just works.  From an implementation perspective, though, the KIP is not so straightforward; a description of how it works therefore follows below.  Note that this description applies to the implementation only – none of this is part of the public API.

This implementation works at a very low level in the Kafka stack, at the level of the network code.  It is therefore transparent to all clients – it just works with no knowledge or accommodation required on their part.  When a client makes a request to a broker the request is intercepted at the level of the Selector class and a check is done to see if re-authentication is enabled; if it is, and the broker supports re-authentication, then the connection is re-authenticated at that point before the request is allowed to flow through.  The solution is elegant because it re-uses existing code paths while requiring no code changes higher up in the stack.

This KIP transparently adds re-authentication support for all uses, which at this point includes the following:

  • org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient
    • org.apache.kafka.clients.consumer.KafkaConsumer
    • org.apache.kafka.connect.runtime.distributed.WorkerGroupMember
  • kafka.controller.ControllerChannelManager
  • org.apache.kafka.clients.admin.KafkaAdminClient
  • org.apache.kafka.clients.producer.KafkaProducer
  • kafka.coordinator.transaction.TransactionMarkerChannelManager
  • kafka.server.ReplicaFetcherBlockingSend (kafka.server.ReplicaFetcherThread)
  • kafka.admin.AdminClient
  • kafka.tools.ReplicaVerificationTool
  • kafka.server.KafkaServer

  • org.apache.kafka.trogdor.workload.ConnectionStressWorker

The final issue to describe is how/when a KafkaChannel instance (each of which corresponds to a unique network connection) is told to re-authenticate.  Each KafkaChannel instance will remember the session expiration time communicated during (re-)authentication (if any); the code in the Selector class will check to see if that time has passed and will start the re-authentication process if so.

Compatibility, Deprecation, and Migration Plan
Anchor
MigrationPlan
MigrationPlan

With respect to compatibility, there is no impact to existing installations because the default is for the server-side connection kill feature to be turned off, older clients never try to re-authenticate because they don't support it, and newer clients that connect to older brokers will know that the broker does not support re-authentication and will therefore not attempt it.

With respect to migration, the approach would be as follows:

  1. Upgrade all brokers to v2.2.0 or later at whatever rate is desired with 'connections.max.reauth.ms' allowed to default to 0.  If SASL is used for the inter-broker protocol then brokers will check the SASL_AUTHENTICATE API version and use a V1 request when communicating to a broker that has been upgraded to 2.2.0, but the client will see the "0" session max lifetime and will not re-authenticate.  Their connections will not be killed.
  2. In parallel with (1) above, upgrade non-broker clients to v2.2.0 or later at whatever rate is desired.  SASL clients will check the SASL_AUTHENTICATE API version and use a V1 request when communicating to a broker that has been upgraded to 2.2.0, but the client will see the "0" session max lifetime and will not re-authenticate.  Their connections will not be killed.
  3. After (1) and (2) are complete, perform a rolling restart of all brokers and check the metrics successful-authentication-no-reauth-{rate,total} to confirm that they remain at zero.  This gives confidence that (1) and (2) are indeed complete.
  4. Update 'connections.max.reauth.ms' to a positive value and perform a rolling restart of brokers again. 
  5. Monitor the successful-authentication-no-reauth-{rate,total} metrics – they will remain at 0 unless an older client connects to the broker.

Rejected Alternatives

Delaying Support for Brokers Killing Connections

It was initially proposed that we defer adding the ability for brokers to kill connections using expired credentials to a future KIP.  This functionality is actually easier to add than re-authentication, and re-authentication without this feature doesn't really improve security (because it can't be enforced).  Adding the ability to kill connections using an expired bearer token without the ability for the client to re-authenticate also does not make sense as a general feature – it forces the client to essentially "recover" from what looks like a network error on a periodic basis.  So we will implement both features at the same time.

Delaying Support for non-OAUTHBEARER SASL Mechanisms

It was initially proposed that we defer adding the ability for SASL clients to re-authenticate when using a non-OAUTHBEARER mechanism (e.g. PLAIN, GSSAPI, and SCRAM-related).  We were able to identify how all mechanisms could be readily and easily supported.

Creating a Configuration Option to Disable Client-side Re-Authentication

It was initially proposed that we would create a client-side configuration option to disable the use of re-authentication on the client.  This may have been necessary when we were contemplating not including support for non-OAUTHBEARER SASL mechanisms and/or when we had not decided to bump the SASL_AUTHENTICATE version number, but it because unnecessary given these decisions.  There is no need to disable the feature if the client and the server both support it.

Validating the Token Lifetime as Part of Re-Authentication

It was initially proposed that we would have the broker reject (re-)authentications that occurred with a credential having a lifetime longer than the maximum allowed.  This was decided to be unnecessary here because the same thing can be done as part of token validation.

Highest-Level Approach: Inserting Requests into Clients' Queues

The original implementation injected requests directly into both asynchronous and synchronous I/O clients' queues.  This implementation was too high up in the stack and required significantly more work and code than any other solution.  It also was much harder to maintain because it became entwined in the implementation of every client.

High-Level Approach: One-Size-Fits-All With Additional KafkaClient Methods

One implementation injected requests into the KafkaClient via a new method (shown below).  This one-size-fits-all approach worked as long as synchronous I/O clients periodically checked for and sent injected requests related to re-authentication (via a new method, also shown below).  This implementation was harder to maintain than the chosen approach because it crossed existing module boundaries related to security code, networking code, and code higher up in the stack – it imposed requirements (however slight) on code higher up in the stack in order for re-authentication to work.  This violation of existing modularity caused concern.  The code was also 2-3 times bigger (at least) relative to the accepted implementation, and it was incrementally (though not dramatically) more difficult to test.  It did create the possibility of interleaving requests related to re-authentication with requests that the client was otherwise sending, which minimized latency spikes due to re-authentication, but that advantage was difficult to quantify and therefore did not tip the balance in favor of this option.

Code Block
languagejava
titleorg.apache.kafka.clients.KafkaClient additions
    /**
     * Return true if any node has a re-authentication request either enqueued and
     * waiting to be sent or already in-flight. A call to {@link #poll(long, long)}
     * is required to send and receive/process the results of such requests. <b>An
     * owner of this instance that does not implement a run loop to repeatedly call
     * {@link #poll(long, long)} but instead only sends requests synchronously
     * on-demand to a single node must call this method periodically -- and invoke
     * {@link #poll(long, long)} if the return value is {@code true} -- to ensure
     * that any re-authentication requests that have been injected are sent and
     * processed in a timely fashion.</b>
     * <p>
     * Example code to re-authenticate a connection across several
     * requests/responses is as follows:
     * 
     * <pre>
     * // Send multiple requests related to re-authentication in the synchronous
     * // use case, completing the re-authentication exchange.
     * while (kafkaClient.hasReauthenticationRequest())
     *     // Returns an empty list in synchronous use case.
     *     kafkaClient.poll(Long.MAX_VALUE, time.milliseconds());
     * // The connection is ready for use, and if there originally was a
     * // re-authentication request then as many requests as required to

...

It was mentioned above that the design supports adding re-authentication to PLAIN, GSSAPI, and SCRAM-related SASL mechanisms but that doing so is out of scope for this KIP.  It is helpful to describe how this could be done in a bit more detail.

There are two requirements: the credential that the Login mechanism adds to the Subject's private credentials on the client side must implement the ExpiringCredential interface; and the client and server sides have to agree on the expiration time of each credential so that the client will know when to refresh the credential and the server will know when the presented credential expires.  The new broker configuration 'connections.max.reauth.ms' takes care of the second requirement.

Implementing the ExpiringCredential interface is pretty easily done by wrapping the data that the LoginModule currently creates and adjusting the SASL Client callback handler accordingly.  For example, the org.apache.kafka.common.security.plain.PlainLoginModule class currently adds a String to the private credentials for the password and a String to the public credentials for the username; it would instead store an instance of a class that implements ExpiringCredential and includes an additional password() method, and the org.apache.kafka.common.security.authenticator.SaslClientCallbackHandler class would be amended to pull the data from that object if it exists (rather than from the pair of String values, which it would fall back to if no ExpiringCredential existed).  Similar adjustments could be done for the SCRAM-related mechanisms as well.  The GSSAPI mechanism would be a bit different because we don't control all the code (we reuse an existing LoginModule implementation, for example); in such cases we would simply use our own implementation that delegates to existing one and that augments where necessary (e.g. in the commit() method of the LoginModule implementation).

...

With respect to compatibility, there is no impact to existing installations because the default is for the feature to be completely turned off on both the client and server.

With respect to migration, the approach would be as follows:

1) Upgrade all brokers to v2.1.0 or later

2) After (1) is complete, turn on re-authentication for brokers (as inter-broker clients, via 'sasl.login.refresh.reauthenticate.enable') at whatever rate is desired -- just eventually, at some point, get the client-side feature turned on for all brokers so that inter-broker connections are re-authenticating. (Skip this step and consider it complete if SASL/OAUTHBEARER is not used for inter-broker communication.)

3) After (2) is complete, partially enable the server-side kill functionality with a negative value for '[listener].oauthbearer.connections.max.reauth.ms' on all brokers.  The metric documenting the number of API requests made over expired connections will begin to increase until the next step (4) is completed.  No connections will be killed.

4) In parallel with (1), (2), and (3) above, upgrade non-broker clients to v2.1.0 or later and turn their re-authentication feature on.  Clients using SASL/OAUTHBEARER will check the API version and only re-authenticate to a broker that has been upgraded to 2.1.0 or later (note that the ability of a broker to respond to a re-authentication cannot be turned off -- it is always on beginning with version 2.1.0, and it just sits there doing nothing if it isn't exercised by an enabled client).

5) After (3) and (4) are complete, check the broker metric documenting the number of API requests made over expired connections to confirm that it is no longer increasing.  Once you are satisfied that (1), (2), (3), and (4) are indeed complete you can fully enable the server-side expired-connection-kill feature on each broker by changing the '[listener].oauthbearer.connections.max.reauth.ms' value from its negative value to its absolute value and restarting the broker.

6) Monitor the metric that documents the number of killed connections – it will remain at 0 unless an older client or one that does not have re-authentication enabled connects to the broker via the SASL/OAUTHBEARER mechanism.

Rejected Alternatives

Delaying Support for Brokers Killing Connections

It was initially proposed that we defer adding the ability for brokers to kill connections using expired credentials to a future KIP.  This functionality is actually easier to add than re-authentication, and re-authentication without this feature doesn't really improve security (because it can't be enforced).  Adding the ability to kill connections using an expired bearer token without the ability for the client to re-authenticate also does not make sense as a general feature – it forces the client to essentially "recover" from what looks like a network error on a periodic basis.  So we will implement both features at the same time.

Highest-Level Approach: Inserting Requests into Clients' Queues

The original implementation injected requests directly into both asynchronous and synchronous I/O clients' queues.  This implementation was too high up in the stack and required significantly more work and code than any other solution.  It also was much harder to maintain because it became entwined in the implementation of every client.

High-Level Approach: One-Size-Fits-All With Additional KafkaClient Methods

One implementation injected requests into the KafkaClient via a new method (shown below).  This one-size-fits-all approach worked as long as synchronous I/O clients periodically checked for and sent injected requests related to re-authentication (via a new method, also shown below).  This implementation was harder to maintain than the chosen approach because it crossed existing module boundaries related to security code, networking code, and code higher up in the stack – it imposed requirements (however slight) on code higher up in the stack in order for re-authentication to work.  This violation of existing modularity caused concern.  The code was also 2-3 times bigger (at least) relative to the accepted implementation, and it was incrementally (though not dramatically) more difficult to test.  It did create the possibility of interleaving requests related to re-authentication with requests that the client was otherwise sending, which minimized latency spikes due to re-authentication, but that advantage was difficult to quantify and therefore did not tip the balance in favor of this option.

Code Block
languagejava
titleorg.apache.kafka.clients.KafkaClient additions
    /**
     * Return true if any node has a re-authentication request either enqueued and
     * waiting to be sent or already in-flight. A call to {@link #poll(long, long)}
     * is required to send and receive/process the results of such requests. <b>An
     * owner of this instance that does not implement a run loop to repeatedly call
     * {@link #poll(long, long)} but instead only sends requests synchronously
     * on-demand to a single node must call this method periodically -- and invoke
     * {@link #poll(long, long)} if the return value is {@code true} -- to ensure
     * that any re-authentication requests that have been injected are sent and
     * processed in a timely fashion.</b>
     * <p>
     * Example code to re-authenticate a connection across several
     * requests/responses is as follows:
     * 
     * <pre>
     * // Sendcomplete multiplethe requestsexchange relatedhave to re-authentication in the synchronousbeen sent.
     * <//pre>
 use case, completing the re-authentication* exchange.
     * while (kafkaClient.hasReauthenticationRequest())
     *     // Returns an empty list in synchronous use case.
     *     kafkaClient.poll(Long.MAX_VALUE, time.milliseconds());Alternatively, to only send one re-authentication request and receive its
     * response (which allows us to interleave other requests to the single node to
     * //which Thewe connectionare isconnected readybefore forsubsequent use,requests andrelated ifto there originally was athe multi-step
     * // re-authentication request then as many requests as required to exchange are sent):
     * 
     * <pre>
     * // complete the exchange have been sent.Send a single request related to re-authentication in the synchronous
     * </pre>
     * // use case, potentially (but not necessarily) completing the
     * Alternatively, to only send one re-authentication request and receive its // re-authentication exchange.
     * while (kafkaClient.hasReauthenticationRequest()) {
     * response (which allows us to// interleaveReturns otheran requestsempty tolist thein singlesynchronous nodeuse tocase.
     * which we are connected before subsequent requests related to the multi-step     kafkaClient.poll(Long.MAX_VALUE, time.milliseconds());
     *     if (!kafkaClient.hasInFlightRequests())
     * re-authentication exchange are sent):
     *          break; // Response has been received.
     * <pre>}
     * // SendThe connection ais singleready requestfor relateduse, toand re-authenticationif inthere thewas synchronousa
     * // use case, potentially (but not necessarily) completing there-authentication request then either the exchange is finished or
     * // there is another re-authentication exchange.
     * while (kafkaClient.hasReauthenticationRequest()) {authentication request available to be sent.
     * </pre>
    // Returns* an
 empty list in synchronous use* case.
@return if any node has *a re-authentication request either  kafkaClient.poll(Long.MAX_VALUE, time.milliseconds());enqueued and
     *     if (!kafkaClient.hasInFlightRequests())
     * waiting to be sent or already in-flight
  break; // Response has* been received.@see #enqueueAuthenticationRequest(ClientRequest)
     * }/
    default boolean hasReauthenticationRequest() {
     * // The connectionreturn isfalse;
 ready for use, and if there was a }


    /**
     * // re-authenticationEnqueue the given request thenrelated either the exchange is finished orto re-authenticating a connection. This
     * // theremethod is another re-authentication request available guaranteed to be sent.
     * </pre>thread-safe even if the class implementing this
     * 
interface is generally not.
  * @return if any* node
 has a re-authentication request either* enqueued@param andclientRequest
     *         waiting to be sentthe orrequest alreadyto in-flightenqueue
     * @see #enqueueAuthenticationRequest#hasReauthenticationRequest(ClientRequest)
     */
    default booleanvoid hasReauthenticationRequestenqueueAuthenticationRequest(ClientRequest clientRequest) {
        return// false;empty
    }


    /**
     * Enqueue the given request related to re-authenticating a connection. This
     * method is guaranteed to be thread-safe even if the class implementing this
     * interface is generally not.
     * 
     * @param clientRequest
     *            the request to enqueue
     * @see #hasReauthenticationRequest()
     */
    default void enqueueAuthenticationRequest(ClientRequest clientRequest) {
        // empty
    }

Adding an ExpiringCredential Public API

It was initially proposed that we make an existing, non-public ExpiringCredential interface part of the public API and leverage the background login refresh thread's refresh event to kick-start re-authentication on the client side for the refreshed credential.  This is unnecessary due to a couple of factors.  First, the server (broker) indicates to the client what the expiration time is, and the low-level mechanism we have chosen on the client side can insert itself into the flow at the correct time – it does not need an external mechanism; and second, the server will chose the token expiration time as the session expiration time if does not exceed the maximum allowable value, which means the refresh thread on the client side will have already refreshed the token (or, if it hasn't, the client can't make new connections anyway).  We had at one time considered that the server rejecting tokens whose remaining lifetime exceeds the maximum allowable session time was a third factor, but that functionality was rejected because it can be done as part of token validation as mentioned above.

Authenticating a Separate Connection and Transferring Credentials

...