KAFKA-18334: Produce v4-v6 should be undeprecated (#18288)

Librdkafka totally breaks if produce v3 is removed - it starts sending records with record format v0.
These api versions have to be undeprecated - KIP-896 has been updated.

Reviewers: David Arthur <mumrah@gmail.com>
This commit is contained in:
Ismael Juma 2024-12-20 16:59:51 -08:00 committed by GitHub
parent 8c21857877
commit 288d4de834
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
2 changed files with 2 additions and 2 deletions

View File

@ -43,7 +43,7 @@
// AddPartitionsToTxn call. If V2 is disabled, the client can't use produce request version higher than 11 within
// a transaction.
"validVersions": "0-12",
"deprecatedVersions": "0-6",
"deprecatedVersions": "0-2",
"flexibleVersions": "9+",
"fields": [
{ "name": "TransactionalId", "type": "string", "versions": "3+", "nullableVersions": "3+", "default": "null", "entityType": "transactionalId",

View File

@ -346,7 +346,7 @@ class SocketServerTest {
assertTrue(requestsPerSec(requestVersion).getOrElse(0L) > 0, "RequestsPerSec should be higher than 0")
assertEquals(None, deprecatedRequestsPerSec(requestVersion))
requestVersion = 3
requestVersion = 2
sendRequest(plainSocket, producerRequestBytes(requestVersion))
receivedReq = receiveRequest(server.dataPlaneRequestChannel)
server.dataPlaneRequestChannel.sendNoOpResponse(receivedReq)