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

KAFKA-18269: Remove deprecated protocol APIs support (KIP-896, KIP-724) #18218

Merged
merged 21 commits into from
Dec 21, 2024

Conversation

ijuma
Copy link
Member

@ijuma ijuma commented Dec 16, 2024

Included in this change:

  1. Remove deprecated protocol api versions from json files.
  2. Remove fields that are no longer used from json files (affects ListOffsets, OffsetCommit, DescribeConfigs).
  3. Remove record down-conversion support from KafkaApis.
  4. No longer return Errors.UNSUPPORTED_COMPRESSION_TYPE on the fetch path[1].
  5. Deprecate TopicConfig. MESSAGE_DOWNCONVERSION_ENABLE_CONFIG and made the relevant
    configs (message.downconversion.enable and log.message.downcoversion.enable) no-ops since
    down-conversion is no longer supported. It was an oversight not to deprecate this via KIP-724.
  6. Fix shouldRetainsBufferReference to handle null request schemas for a given version.
  7. Simplify producer logic since it only supports the v2 record format now.
  8. Fix tests so they don't exercise protocol api versions that have been removed.
  9. Add upgrade note.

Testing:

  1. System tests have a lot of failures, but those tests fail for trunk too and I didn't see any issues specific to this change - it's hard to be sure given the number of failing tests, but let's not block on that given the other testing that has been done (see below).
  2. Java producers and consumers with version 0.9-0.10.1 don't have api versions support and hence they fail in an ungraceful manner: the broker disconnects and the clients reconnect until the relevant timeout is triggered.
  3. Same thing seems to happen for the console producer 0.10.2 although it's unclear why since api versions should be supported. I will look into this separately, it's unlikely to be related to this PR.
  4. Console consumer 0.10.2 fails with the expected error and a reasonable message[2].
  5. Console producer and consumer 0.11.0 works fine, newer versions should naturally also work fine.
  6. kcat 1.5.0 (based on librdkafka 1.1.0) produce and consume fail with a reasonable message[3][4].
  7. kcat 1.6.0-1.7.0 (based on librdkafka 1.5.0 and 1.7.0 respectively) consume fails with a reasonable message[5].
  8. kcat 1.6.0-1.7.0 produce works fine.
  9. kcat 1.7.1 (based on librdkafka 1.8.2) works fine for consumer and produce.
  10. confluent-go-client (librdkafka based) 1.8.2 works fine for consumer and produce.
  11. I will test more clients, but I don't think we need to block the PR on that.

Note that this also completes part of KIP-724: produce v2 and lower as well as fetch v3 and lower are no longer supported.

Future PRs will remove conditional code that is no longer needed (some of that has been done in KafkaApis,
but only what was required due to the schema changes). We can probably do that in master only as it does
not change behavior.

Note that I did not touch ignorable fields even though some of them could have been
changed. The reasoning is that this could result in incompatible changes for clients
that use new protocol versions without setting such fields if we don't manually
validate their presence. I will file a JIRA ticket to look into this carefully for each
case (i.e. if we do validate their presence for the appropriate versions, we can
set them to ignorable=false in the json file).

[1] We would return this error if a fetch < v10 was used and the compression topic config was set
to zstd, but we would not do the same for the case where zstd was compressed at the producer
level (the most common case). Since there is no efficient way to do the check for the common
case, I made it consistent for both by having no checks.
[2] org.apache.kafka.common.errors.UnsupportedVersionException: The broker is too new to support JOIN_GROUP version 1
[3]METADATA|rdkafka#producer-1| [thrd:main]: localhost:9092/bootstrap: Metadata request failed: connected: Local: Required feature not supported by broker (0ms): Permanent
[4]METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9092/bootstrap: Metadata request failed: connected: Local: Required feature not supported by broker (0ms): Permanent
[5] ERROR: Topic test-topic [0] error: Failed to query logical offset END: Local: Required feature not supported by broker

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

@mumrah
Copy link
Member

mumrah commented Dec 16, 2024

@ijuma just so you know, the JUnit tests won't run on draft PRs.

@ijuma ijuma marked this pull request as ready for review December 16, 2024 19:14
@ijuma
Copy link
Member Author

ijuma commented Dec 16, 2024

Thanks for the heads up @mumrah - super helpful.

This is not yet ready for review, I would like to see if there are any failures before that (I ran a subset and they passed locally). But I changed the status so that the tests run.

@ijuma ijuma changed the title KAFKA-XXX: Remove deprecated protocol APIs support (KIP-896) KAFKA-18269: Remove deprecated protocol APIs support (KIP-896) Dec 16, 2024
@ijuma ijuma force-pushed the kip-896-remove-old-protocol-versions branch from 59b1e19 to c790b59 Compare December 16, 2024 20:07
@ijuma ijuma force-pushed the kip-896-remove-old-protocol-versions branch from c790b59 to a8842c7 Compare December 17, 2024 08:40
@github-actions github-actions bot added the storage Pull requests that target the storage module label Dec 17, 2024
@ijuma ijuma changed the title KAFKA-18269: Remove deprecated protocol APIs support (KIP-896) KAFKA-18269: Remove deprecated protocol APIs support (KIP-896, KIP-724) Dec 18, 2024
@github-actions github-actions bot removed the triage PRs from the community label Dec 18, 2024
@ijuma ijuma force-pushed the kip-896-remove-old-protocol-versions branch from 5c22c4e to 3d83afc Compare December 18, 2024 15:40
@ijuma ijuma requested a review from mumrah December 18, 2024 16:00
Copy link
Member

@mumrah mumrah left a comment

Choose a reason for hiding this comment

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

Took a pass through the rest of the PR. A few comments inline. Overall, I think it looks good. I think we'll be leaning on the test suite for this change :)

Will we be able to delete LazyDownConversionRecords and the remaining down conversion code after this PR lands?

@@ -42,27 +42,16 @@
public class ProduceRequest extends AbstractRequest {
public static final short LAST_STABLE_VERSION_BEFORE_TRANSACTION_V2 = 11;

public static Builder forMagic(byte magic, ProduceRequestData data, boolean useTransactionV1Version) {
Copy link
Member

Choose a reason for hiding this comment

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

Nice to see this record version stuff going away 😄

@ijuma
Copy link
Member Author

ijuma commented Dec 18, 2024

Will we be able to delete LazyDownConversionRecords and the remaining down conversion code after this PR lands?

Yes. This and a few other things are what I meant by:

Future PRs will remove conditional code that is no longer needed (some of that has been done in KafkaApis,
but only what was required due to the schema changes). We can probably do that in master only as it does
not change behavior.

I think we'll be leaning on the test suite for this change :)

Yeah, I think that will ensure the remaining protocol versions work correctly. For the clients that are no longer supported, I will test them manually and considering what we can do via the system tests.

…e-old-protocol-versions

* apache-github/trunk: (25 commits)
  KAFKA-18270: FindCoordinator v0 incorrectly tagged as deprecated (apache#18262)
  KAFKA-18284: Add group coordinator records for Streams rebalance protocol (apache#18228)
  MINOR: Fix flaky state updater test (apache#18253)
  MINOR: improve StreamsResetter logging (apache#18237)
  KAFKA-18227: Ensure v2 partitions are not added to last transaction during upgrade (apache#18176)
  Add IT for share consumer with duration base offet auto reset (apache#18251)
  KAFKA-18283: Add StreamsGroupDescribe RPC definitions (apache#18230)
  KAFKA-18241: add docs check to CI (apache#18183)
  KAFKA-18223 Improve flaky test report (apache#18212)
  MINOR Remove triage label in nightly job (apache#18147)
  KAFKA-18294 Remove deprecated SourceTask#commitRecord (apache#18260)
  KAFKA-18264 Remove NotLeaderForPartitionException (apache#18211)
  KAFKA-13722: Refactor SerdeGetter (apache#18242)
  KAFKA-18094 Remove deprecated TopicListing(String, Boolean) (apache#18248)
  KAFKA-18282: Add StreamsGroupHeartbeat RPC definitions (apache#18227)
  KAFKA-18026: KIP-1112 migrate KTableSuppressProcessorSupplier (apache#18150)
  KAFKA-18026: transition KTable#filter impl to use processor wrapper (apache#18205)
  KAFKA-18293 Remove `org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler` and `org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerValidatorCallbackHandler` (apache#18244)
  MINOR: add assertion about groupEpoch and targetAssignmentEpoch to testConsumerGroups (apache#18203)
  KAFKA-17960; PlaintextAdminIntegrationTest.testConsumerGroups fails with CONSUMER group protocol (apache#18234)
  ...
@ijuma
Copy link
Member Author

ijuma commented Dec 19, 2024

@mumrah I merged master to the branch, fixed the FIXMEs (simply removed them) and updated the PR description with the testing I have done so far and what's in progress. I expect the PR is mostly ready, the only thing I intend to do before merging is fixing any system test failures due to this PR. Let me know if you think there's anything else that needs to be done here.

…e-old-protocol-versions

* apache-github/trunk:
  KAFKA-18312: Added entityType: topicName to SubscribedTopicNames in ShareGroupHeartbeatRequest.json (apache#18285)
  HOTFIX: fix incompatible types: Optional<TimestampAndOffset> cannot be converted to Option<TimestampAndOffset> (apache#18284)
  MINOR Fix some test-catalog issues (apache#18272)
  KAFKA-18180: Move OffsetResultHolder to storage module (apache#18100)
  KAFKA-18301; Make coordinator records first class citizen (apache#18261)
  KAFKA-18262 Remove DefaultPartitioner and UniformStickyPartitioner (apache#18204)
  KAFKA-18296 Remove deprecated KafkaBasedLog constructor (apache#18257)
  KAFKA-12829: Remove old Processor and ProcessorSupplier interfaces (apache#18238)
  KAFKA-18292 Remove deprecated methods of UpdateFeaturesOptions (apache#18245)
  KAFKA-12829: Remove deprecated Topology#addProcessor of old Processor API (apache#18154)
  KAFKA-18035, KAFKA-18306, KAFKA-18092: Address TransactionsTest flaky tests (apache#18264)
  MINOR: change the default linger time in the new coordinator (apache#18274)
  KAFKA-18305: validate controller.listener.names is not in inter.broker.listener.name for kcontrollers (apache#18222)
  KAFKA-18207: Serde for handling transaction records (apache#18136)
  KAFKA-13722: Refactor Kafka Streams store interfaces (apache#18243)
  KAFKA-17131: Refactor TimeDefinitions (apache#18241)
  MINOR: Fix MessageFormatters (apache#18266)
  Mark flaky tests for Dec 18, 2024 (apache#18263)
@ijuma
Copy link
Member Author

ijuma commented Dec 20, 2024

@mumrah I think this is ready to go. I updated the PR description with the client versions I tested. One of the consequences of the testing was KAFKA-18334. I adjusted this PR to take it into account and submitted #18288 as well.

Copy link
Member

@mumrah mumrah 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 the work on this @ijuma, LGTM.

Will you merge #18288 in first?

@ijuma
Copy link
Member Author

ijuma commented Dec 20, 2024

Yes @mumrah, that's the plan. And then merge trunk to this PR and then merge this PR once green.

…e-old-protocol-versions

* apache-github/trunk:
  KAFKA-18334: Produce v4-v6 should be undeprecated (apache#18288)
  KAFKA-13722: code cleanup after deprecated StateStore.init() was removed (apache#18249)
  KAFKA-15370: Support Participation in 2PC (KIP-939) (1/N) (apache#17687)
@ijuma ijuma merged commit fe56fc9 into apache:trunk Dec 21, 2024
9 checks passed
@ijuma ijuma deleted the kip-896-remove-old-protocol-versions branch December 21, 2024 04:02
@frankvicky
Copy link
Contributor

Hi @ijuma,

I noticed a discrepancy between this PR and KIP-896 regarding DeleteGroups support:

DeleteGroups: none due to clients (Sarama) support - V0 per the baseline
* librdkafka: [V1](https://github.com/confluentinc/librdkafka/blob/v1.8.2/src/rdkafka_request.c#L4147)
* KafkaJS: [V1](https://github.com/tulios/kafkajs/blob/v1.15.0/src/protocol/requests/deleteGroups/v1/request.js#L7)
* Sarama: [V0](https://github.com/Shopify/sarama/blob/v1.29.1/describe_groups_request.go#L21)
* kafka-python: [V1](https://github.com/dpkp/kafka-python/blob/2.0.2/kafka/admin/client.py#L1328)

According to the KIP, Sarama only supports V0 of the DeleteGroups RPC. However, I found issue (IBM/sarama#2408) which suggests Sarama will upgrade its RPC support. (And it has support v1 after IBM/sarama#2551) If this is the case, should we update the description in KIP-896 (https://cwiki.apache.org/confluence/x/K5sODg)?

WDYT?

@ijuma
Copy link
Member Author

ijuma commented Jan 3, 2025

No, that's not a discrepancy as far as I can tell. The KIP specifies a particular version of Sarama and links to the relevant code. Can you please check again with that in mind?

@chia7712
Copy link
Member

chia7712 commented Jan 3, 2025

@frankvicky the v0 version is already added back. Please see 2e3b4cb

@frankvicky
Copy link
Contributor

Oh, I should check trunk first.
Thanks for pointing out 🙇🏼

@ijuma
Copy link
Member Author

ijuma commented Jan 3, 2025

Thanks for checking - it's helpful to make sure!

tedyu pushed a commit to tedyu/kafka that referenced this pull request Jan 6, 2025
…4) (apache#18218)

Included in this change:
1. Remove deprecated protocol api versions from json files.
3. Remove fields that are no longer used from json files (affects ListOffsets, OffsetCommit, DescribeConfigs).
4. Remove record down-conversion support from KafkaApis.
5. No longer return `Errors.UNSUPPORTED_COMPRESSION_TYPE` on the fetch path[1].
6. Deprecate `TopicConfig. MESSAGE_DOWNCONVERSION_ENABLE_CONFIG` and made the relevant
configs (`message.downconversion.enable` and `log.message.downcoversion.enable`) no-ops since
down-conversion is no longer supported. It was an oversight not to deprecate this via KIP-724.
7. Fix `shouldRetainsBufferReference` to handle null request schemas for a given version.
8. Simplify producer logic since it only supports the v2 record format now.
9. Fix tests so they don't exercise protocol api versions that have been removed.
10. Add upgrade note.

Testing:
1. System tests have a lot of failures, but those tests fail for trunk too and I didn't see any issues specific to this change - it's hard to be sure given the number of failing tests, but let's not block on that given the other testing that has been done (see below).
3. Java producers and consumers with version 0.9-0.10.1 don't have api versions support and hence they fail in an ungraceful manner: the broker disconnects and the clients reconnect until the relevant timeout is triggered.
4. Same thing seems to happen for the console producer 0.10.2 although it's unclear why since api versions should be supported. I will look into this separately, it's unlikely to be related to this PR.
5. Console consumer 0.10.2 fails with the expected error and a reasonable message[2].
6. Console producer and consumer 0.11.0 works fine, newer versions should naturally also work fine.
7. kcat 1.5.0 (based on librdkafka 1.1.0) produce and consume fail with a reasonable message[3][4].
8. kcat 1.6.0-1.7.0 (based on librdkafka 1.5.0 and 1.7.0 respectively) consume fails with a reasonable message[5].
9. kcat 1.6.0-1.7.0 produce works fine.
10. kcat 1.7.1  (based on librdkafka 1.8.2) works fine for consumer and produce.
11. confluent-go-client (librdkafka based) 1.8.2 works fine for consumer and produce.
12. I will test more clients, but I don't think we need to block the PR on that.

Note that this also completes part of KIP-724: produce v2 and lower as well as fetch v3 and lower are no longer supported.

Future PRs will remove conditional code that is no longer needed (some of that has been done in KafkaApis,
but only what was required due to the schema changes). We can probably do that in master only as it does
not change behavior.

Note that I did not touch `ignorable` fields even though some of them could have been
changed. The reasoning is that this could result in incompatible changes for clients
that use new protocol versions without setting such fields _if_ we don't manually
validate their presence. I will file a JIRA ticket to look into this carefully for each
case (i.e. if we do validate their presence for the appropriate versions, we can
set them to ignorable=false in the json file).

[1] We would return this error if a fetch < v10 was used and the compression topic config was set
to zstd, but we would not do the same for the case where zstd was compressed at the producer
level (the most common case). Since there is no efficient way to do the check for the common
case, I made it consistent for both by having no checks.
[2] ```org.apache.kafka.common.errors.UnsupportedVersionException: The broker is too new to support JOIN_GROUP version 1```
[3]```METADATA|rdkafka#producer-1| [thrd:main]: localhost:9092/bootstrap: Metadata request failed: connected: Local: Required feature not supported by broker (0ms): Permanent```
[4]```METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9092/bootstrap: Metadata request failed: connected: Local: Required feature not supported by broker (0ms): Permanent```
[5] `ERROR: Topic test-topic [0] error: Failed to query logical offset END: Local: Required feature not supported by broker`

Reviewers: David Arthur <[email protected]>
ijuma added a commit that referenced this pull request Jan 29, 2025
During testing, we identified that kafka-python (and aiokafka) relies on metadata request v0 and
hence we need to add these back to comply with the premise of KIP-896 - i.e. it should not
break the clients listed within it.

I reverted the changes from #18218 related to the removal of metadata versions 0-3.

I will submit a separate PR to undeprecate these API versions on the relevant 3.x branches.

kafka-python (and aiokafka) work correctly (produce & consume) with this change on
top of the 4.0 branch.

Reviewers: David Arthur <[email protected]>
ijuma added a commit that referenced this pull request Jan 29, 2025
During testing, we identified that kafka-python (and aiokafka) relies on metadata request v0 and
hence we need to add these back to comply with the premise of KIP-896 - i.e. it should not
break the clients listed within it.

I reverted the changes from #18218 related to the removal of metadata versions 0-3.

I will submit a separate PR to undeprecate these API versions on the relevant 3.x branches.

kafka-python (and aiokafka) work correctly (produce & consume) with this change on
top of the 4.0 branch.

Reviewers: David Arthur <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
clients consumer core Kafka Broker kraft performance producer storage Pull requests that target the storage module
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants