Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add broker reauthentication [KIP-368] #4301

Merged
merged 9 commits into from
Jun 14, 2023
Merged

Conversation

milindl
Copy link
Contributor

@milindl milindl commented May 30, 2023

Thanks @vctoriawu for starting this in #3754 .

This builds up on that and changes it as per the comments, and the internal discussion with Magnus and the rest of the team.

Here's how we're doing the reauthentication:

  1. In case we get a non-zero session_lifetime_ms in the SaslAuthenticate response for a broker rkb, start a timer for that broker at 90% of that.
  2. [main thread] The timer is hit and the callback triggered. The callback enqueues an op on the broker rkb.
  3. [broker thread] We get this op in rd_kafka_broker_op_serve, and we set max_inflight request to 1, and change the broker state into one of reauth.
  4. [broker thread] When we encounter the reauth state in rd_kafka_broker_thread_main, we do some cleanup for the preexisting SASL state, and just do Auth exactly the same as the normal way (when we do it the first time around). This takes care of resetting max_inflight to the correct value, too.

As the KIP and the discussion in #3754 points out, we can't send anything between the auth requests.
Setting max_inflight to 1 means that only one request may be in flight, and since the Sasl* requests have a high priority in the queue (RD_KAFKA_PRIO_FLASH), they will actually hold the other requests back till authentication is complete. Setting it to 1 also means that any requests already in flight will await responses before the auth sequence starts.

For OAUTHBEARER, the token itself has an expiry time. There are two cases here:

  1. broker's connections.max.reauth.ms > time left to token's expiry:
    In this case, the session_lifetime_ms in the SaslAuthenticate response is set to the time left for the token's expiry. Since our OAUTHBEARER callback runs at 80% of (time left to token's expiry) and our reauth runs at 90% of (time left to token's expiry), we'll refresh the token before the reauth.
    It's somewhat trickier than that, because next_token_refresh_time := client_time + 0.8*(token_expiry - client_time) and reauth_time := client_time + 0.9*session_lifetime_ms. Since session_lifetime_ms is calculated on the server, it might have some drift between the clocks, as well as well as discount the time it takes for the communication between the client/server. But it's expected that even if token_expiry - client_time != session_lifetime_ms, the 0.8/0.9 factors will make up for it, as typical token refresh/re-authentication times are on the order of hours (and not seconds).
  2. broker's connections.max.reauth.ms < time left to token's expiry:
    The broker returns connections.max.reauth.ms as the session_lifetime_ms. Since the token has a later expiry, we just use the same token to reauthenticate.

Adds tests (0142. 0141 is already there in the KIP-881)

Also fixes a bug where while using OAUTHBEARER with the unsecure token would not have the callback queue initialized.

Also fixes several bugs with the interactive broker script which caused it to completely ignore any user-provided config, and I added to the documentation as well.

A new parameter has been added to the interactive broker which accepts a reauth time for the broker. The default value of this has been set to a non-zero number. This will serve to provide some extra coverage in long running tests - it is intentionally so.

@milindl milindl requested a review from emasab May 30, 2023 06:03
@milindl milindl requested a review from a team May 30, 2023 08:16
Victoria Wu and others added 6 commits June 14, 2023 10:56
1. On broker fail, reset the timer and the reauth flag.
2. rktrans state should be set to NULL after freeing for all the
   providers.
3. Add a reauth flag to prevent socket_connection_setup_timeout_ms
   related timeouts which should not occur in reauths
4. Test fixes (timeout)
@milindl milindl force-pushed the dev_kip-368-sasl-reauth branch from 68868fd to 318f071 Compare June 14, 2023 05:39
INTRODUCTION.md Show resolved Hide resolved
tests/interactive_broker_version.py Outdated Show resolved Hide resolved
milindl and others added 2 commits June 14, 2023 14:56
@milindl milindl requested a review from emasab June 14, 2023 09:27
src/rdkafka_broker.c Outdated Show resolved Hide resolved
src/rdkafka_sasl_cyrus.c Show resolved Hide resolved
@milindl milindl requested a review from emasab June 14, 2023 10:34
Copy link
Contributor

@emasab emasab left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM!

@milindl milindl merged commit 865cf60 into master Jun 14, 2023
@milindl milindl deleted the dev_kip-368-sasl-reauth branch June 14, 2023 12:43
lpsinger added a commit to lpsinger/gcn-kafka-python that referenced this pull request Aug 23, 2023
This version improves reliability of refreshing the authentication
of long-lived sessions.

See confluentinc/librdkafka#4301.
lpsinger added a commit to nasa-gcn/gcn-kafka-python that referenced this pull request Aug 23, 2023
This version improves reliability of refreshing the authentication
of long-lived sessions.

See confluentinc/librdkafka#4301.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants