-
Notifications
You must be signed in to change notification settings - Fork 63
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
Proposal: Reauthenticate SASL connections based on session lifetime #122
Merged
zmstone
merged 4 commits into
kafka4beam:master
from
salemove:reauthenticate_sasl_connections
Aug 14, 2024
Merged
Proposal: Reauthenticate SASL connections based on session lifetime #122
zmstone
merged 4 commits into
kafka4beam:master
from
salemove:reauthenticate_sasl_connections
Aug 14, 2024
Conversation
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
urmastalimaa
changed the title
Reauthenticate SASL connections based on session lifetime
Proposal: Reauthenticate SASL connections based on session lifetime
Aug 12, 2024
zmstone
reviewed
Aug 13, 2024
urmastalimaa
force-pushed
the
reauthenticate_sasl_connections
branch
3 times, most recently
from
August 13, 2024 10:57
f163a82
to
9f1e8cf
Compare
looks good, please help to add a change log (bump version to |
just merged a ci fix to master, please rebase to make ci run. |
`get_api_vsns` and `{ssl, Sock, Bin}` were missing from print_msg/3 cases and were reported as "unknown messages".
This expansion of the callback return values allows `kpro_connection` to interrogate the server response message, in preparation for re-authenticating SASL connections before session lifetime expires. Authentication was moved to a separate function to allow repeating authentication flow, which also required storing connection configuration in process state.
The broker response to a SASL authentication request can contain a maximum session lifetime (see the [KIP][kip]). Session lifetime is returned by the broker in [Version 1 SaslAuthenticate Response][sasl_authenticate_protocol]. When a SASL authentication callback returns `{ok, ServerResponse}` and the ServerResponse contains a larger than 0 session lifetime, kpro_connection automatically sets a timer to re-authenticate in half the session lifetime. As kpro_sasl mechanisms are synchronous, in-flight requests must first be drained to ensure that kpro_sasl receives a response to its own SASL request. The draining algorithm behaves as follows: * `sasl_authenticate` message handler adds the message onto the backlog and immediately flush the backlog if there are no in-flight requests. * `{From, {send, Request}}` handler adds the request onto the backlog if the backlog has any items to allow in-flight requests to drain. * Inbound message handler flushes the backlog if in-flight requests are empty. [kip]: https://cwiki.apache.org/confluence/display/KAFKA/KIP-368%3A+Allow+SASL+Connections+to+Periodically+Re-Authenticate [sasl_authenticate_protocol]: https://kafka.apache.org/protocol#The_Messages_SaslAuthenticate
urmastalimaa
force-pushed
the
reauthenticate_sasl_connections
branch
from
August 13, 2024 14:15
9f1e8cf
to
49ccc86
Compare
Thanks. Done with diff |
thank you @urmastalimaa |
Perfect, thank you for the fast and excellent feedback. |
9 tasks
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Successor of #119
The broker response to a SASL authentication request can contain a
maximum session lifetime (see the KIP).
Session lifetime is returned by the broker in Version 1
SaslAuthenticate Response.
When a SASL authentication callback returns
{ok, ServerResponse}
andthe ServerResponse contains a larger than 0 session lifetime,
kpro_connection automatically sets a timer to re-authenticate in half
the session lifetime.
As kpro_sasl mechanisms are synchronous, in-flight requests must first
be drained to ensure that kpro_sasl receives a response to its own SASL
request.
The draining mechanism is tied to the main loop, flushing the post-drain
queue when
requests
are empty. When requests are not empty, previousbehaviour is retained with the exception of
{From, {send, Req}}
handler, which adds the request onto the queue when in
drain
state.