Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

Table of Contents

Status

Current stateUnder Discussion

...

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

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 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 clients (and brokers when OAUTHBEARER 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.

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.  Also explicitly out-of-scope for this proposal is the ability for brokers to close connections that continue to use expired credentials.  This ability is a natural next step, but it will be addressed via a separate KIP if/when this one is adopted.

Public Interfaces

This KIP proposes the addition of a single interface to the API and a single additional configuration option to enable the feature (the option value defaults to false, of course, so there is no change to existing behavior in the absence of an explicit opt-in).  Specifically, the interface it proposes to add is as follows:

...

Although there is no technical reason preventing it, we arbitrarily decide to disallow changing identities upon re-authentication.  For example, if a connection originally authenticates as User:user1, an attempt to re-authenticate as anything else (e.g. User:user2) will fail.  Retry is allowed indefinitely in this case.

Proposed Changes

The description of this KIP is actually quite straightforward from a functionality perspective – turn the feature on with the configuration option 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 will work for those mechanisms, too.  From an implementation perspective, though, the KIP is not so straightforward; it therefore includes a pull request with a proposed implementation.  Here is a high-level description of how the proposed implementation generally works.  Note that this description applies to the implementation only – none of this is part of the public API.

...

Finally, we add methods on SaslServerAuthenticator to respond to the requests that arrive related to re-authentication – specifically, respondToReauthenticationSaslHandshakeRequest() and respondToReauthenticationSaslAuthenticateRequest() – and we add code to kafka.server.KafkaApis to route the received requests accordingly (as opposed to responding with an error, which is what currently happens).  I won't go into the details here; see the PR for the code.

Compatibility, Deprecation, and Migration Plan

There is no impact to existing installations because the default is for the feature to be turned off.

Rejected Alternatives

A single queue owned by org.apache.kafka.clients.NetworkClient does not reliably work in the synchronous case as described above; we cannot know how soon an injected request will be sent (if ever).

...