Skip to content

Comments

feat(protocol): negotiate API versions#3209

Merged
dnwe merged 6 commits intoIBM:mainfrom
trapped:centralized-version-negotiation
Jul 11, 2025
Merged

feat(protocol): negotiate API versions#3209
dnwe merged 6 commits intoIBM:mainfrom
trapped:centralized-version-negotiation

Conversation

@trapped
Copy link
Contributor

@trapped trapped commented Jun 27, 2025

This PR introduces basic Kafka API version negotiation in Sarama by enabling the client to restrict the API version for each request based on the version ranges advertised by brokers in the initial ApiVersions request/response - which is now executed synchronously as part of the broker connection process, rather than as a fire-and-forget deferred goroutine, and before SASL authentication takes place (but after SSL/TLS, as indicated by the Kafka protocol docs).

The selected version is the highest supported by both the broker and the user-defined Config.Version upper bound.

To support this dynamic selection, a new setVersion(int16) method has been added to the protocolBody interface, exposing the request/response structs' Version field to be set externally.

Also, it defines constants for message keys, and replaces some (read: those I found) usages of numeric literals to use the constants instead.

I experimented with extracting more logic from protocolBody implementations, too, such as centralizing requiredVersion(), isValidVersion(), and the "max protocol version for Kafka version" selection in most Request constructors as suggested by @dnwe in my previous PR; however, while they can certainly simplify the code, I found these were significantly larger-scale changes (and kinda out of scope for the purpose of this PR), and decided to leave them out to keep this PR short and easy to review.

Motivation behind the change: Sarama incompatibility with Redpanda

The latest github.com/IBM/sarama version fails to connect to Redpanda brokers (latest version too):

  • Redpanda rejects the request with Unsupported version 10 for metadata API
  • sarama fails with kafka: client has run out of available brokers to talk to: EOF
Sarama logs ``` 2025/05/13 18:26:29 Initializing new client 2025/05/13 18:26:29 ClientID is the default of 'sarama', you should consider setting it to something application-specific. 2025/05/13 18:26:29 ClientID is the default of 'sarama', you should consider setting it to something application-specific. 2025/05/13 18:26:29 client/metadata fetching metadata for all topics from broker localhost:9092 2025/05/13 18:26:29 Connected to broker at localhost:9092 (unregistered) 2025/05/13 18:26:29 client/metadata got error from broker -1 while fetching metadata: EOF 2025/05/13 18:26:29 Closed connection to broker localhost:9092 2025/05/13 18:26:29 client/metadata no available broker to send metadata request to 2025/05/13 18:26:29 client/brokers resurrecting 1 dead seed brokers 2025/05/13 18:26:29 Error while sending ApiVersionsRequest to broker localhost:9092: kafka: broker not connected 2025/05/13 18:26:30 client/metadata retrying after 250ms... (2 attempts remaining) 2025/05/13 18:26:30 ClientID is the default of 'sarama', you should consider setting it to something application-specific. 2025/05/13 18:26:30 client/metadata fetching metadata for all topics from broker localhost:9092 2025/05/13 18:26:30 Connected to broker at localhost:9092 (unregistered) 2025/05/13 18:26:30 client/metadata got error from broker -1 while fetching metadata: EOF 2025/05/13 18:26:30 Closed connection to broker localhost:9092 2025/05/13 18:26:30 client/metadata no available broker to send metadata request to 2025/05/13 18:26:30 client/brokers resurrecting 1 dead seed brokers 2025/05/13 18:26:30 client/metadata retrying after 250ms... (1 attempts remaining) 2025/05/13 18:26:30 ClientID is the default of 'sarama', you should consider setting it to something application-specific. 2025/05/13 18:26:30 client/metadata fetching metadata for all topics from broker localhost:9092 2025/05/13 18:26:30 Connected to broker at localhost:9092 (unregistered) 2025/05/13 18:26:30 client/metadata got error from broker -1 while fetching metadata: EOF 2025/05/13 18:26:30 Closed connection to broker localhost:9092 2025/05/13 18:26:30 client/metadata no available broker to send metadata request to 2025/05/13 18:26:30 client/brokers resurrecting 1 dead seed brokers 2025/05/13 18:26:30 client/metadata retrying after 250ms... (0 attempts remaining) 2025/05/13 18:26:30 ClientID is the default of 'sarama', you should consider setting it to something application-specific. 2025/05/13 18:26:30 client/metadata fetching metadata for all topics from broker localhost:9092 2025/05/13 18:26:30 Connected to broker at localhost:9092 (unregistered) 2025/05/13 18:26:30 client/metadata got error from broker -1 while fetching metadata: EOF 2025/05/13 18:26:30 Closed connection to broker localhost:9092 2025/05/13 18:26:30 client/metadata no available broker to send metadata request to 2025/05/13 18:26:30 client/brokers resurrecting 1 dead seed brokers 2025/05/13 18:26:30 Closing Client ```

The culprit being simply that:

  1. Sarama chooses request versions solely according to the provided config.Version, even though it sends an ApiVersionsRequest
  2. MetadataRequest uses versions 8-9-10 for Kafka versions >= v2.4.0.0
  3. Redpanda only supports versions 0 to 8, and it correctly advertises it

@trapped trapped force-pushed the centralized-version-negotiation branch from 3d8a4d2 to bd79dfa Compare June 27, 2025 18:12
trapped added 5 commits June 29, 2025 00:02
Signed-off-by: Giorgio Pellero <giorgio.pellero@typeform.com>
Signed-off-by: Giorgio Pellero <giorgio.pellero@typeform.com>
Signed-off-by: Giorgio Pellero <giorgio.pellero@typeform.com>
Signed-off-by: Giorgio Pellero <giorgio.pellero@typeform.com>
Signed-off-by: Giorgio Pellero <giorgio.pellero@typeform.com>
@trapped trapped force-pushed the centralized-version-negotiation branch from cf8d039 to 6ca2800 Compare June 29, 2025 09:04
@trapped trapped marked this pull request as ready for review June 29, 2025 09:06
Copy link
Collaborator

@puellanivis puellanivis left a comment

Choose a reason for hiding this comment

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

Nothing big.

Signed-off-by: Giorgio Pellero <giorgio.pellero@typeform.com>
@trapped
Copy link
Contributor Author

trapped commented Jul 11, 2025

@puellanivis I've addressed your review comments - would love it if you could have a second look when you've got some time! Thanks 🙂

Copy link
Collaborator

@puellanivis puellanivis left a comment

Choose a reason for hiding this comment

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

Looks pretty good from my side. (note: I do not have permissions to approve PRs.)

Copy link
Collaborator

@dnwe dnwe left a comment

Choose a reason for hiding this comment

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

Thanks for working on this PR. Changes look good to me too so I’ll go ahead and merge into main. I’ll roll the main branch client out to a few places and leave it running for a bit before I cut a release

I’m also wondering about enabling usingApiVersions by default too and making DefaultVersion max too

@dnwe dnwe merged commit 0dd02e1 into IBM:main Jul 11, 2025
17 checks passed
@dnwe dnwe added the feat label Jul 11, 2025
@ijuma
Copy link

ijuma commented Jul 11, 2025

Great to see this, thanks!

@trapped
Copy link
Contributor Author

trapped commented Jul 16, 2025

After some deeper testing with Redpanda (I've found its small differences to be helpful in testing protocol compliance), I've noticed something that I think could use some further improvement in a new PR.

The good:

The bad:

In my testing I could reproduce this issue as follows:

  1. setting config.Version = sarama.MaxVersion
  2. pointing to Redpanda (latest version)
  3. Sarama tries connecting to brokers, negotiating API versions; max version for MetadataRequest is advertised as 8, while the max supported by Sarama is 10
  4. the response promise is initialized with header version 1, corresponding to MetadataResponse v10
  5. the MetadataRequest v8 succeeds, and Redpanda returns a MetadataResponse v8 - header version 0 (flexibleVersions from v9 onwards for MetadataResponse, and we're using v8)
  6. Sarama tries parsing the response payload, starting with the header: because Sarama thinks the header version is v1, it consumes 9 bytes instead of 8
  7. Sarama parses the header successfully and proceeds to read the MetadataResponse body, but misaligned by 1 byte: instead of a 1-element brokers array, it thinks there's 256; instead of a 9-byte host string, it tries reading a 2403-byte string, which fails with "not enough bytes".

Replacing the response promise's headerVersion with a pointer to the response interface, which allows sendInternal() to update its version, could be a potential solution: #3223

Let me know what you think!

@ijuma
Copy link

ijuma commented Jul 20, 2025

The selected version is the highest supported by both the broker and the user-defined Config.Version upper bound.

Have we changed the default for the user-defined Config.Version upper bound? Ideally that would unset by default like most kafka client libraries (i.e. the client should use the highest version supported by the client and the broker).

@dnwe
Copy link
Collaborator

dnwe commented Jul 20, 2025

@ijuma yep that is not included in this PR, for staging purposes, but is the intention before cutting the next release. Currently trialling it via the FV in #3223

@ijuma
Copy link

ijuma commented Jul 20, 2025

Makes sense, thanks.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants