Commit Graph

5638 Commits

Author SHA1 Message Date
Ken Huang ce5d7be1dc
KAFKA-18492 Cleanup RequestHandlerHelper (#18608)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-19 18:41:41 +08:00
Ken Huang bb3944c1d0
KAFKA-18427: Remove ZooKeeperClient (#18613)
Reviewers: Ismael Juma <ismael@juma.me.uk, TingIāu "Ting" Kì <51072200+frankvicky@users.noreply.github.com>
2025-01-18 21:14:46 -08:00
TaiJuWu 4772805cd5
KAFKA-18540: Remove `UpdataMetadataRequest` from `KafkaApisTest` (#18591)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2025-01-18 21:12:52 -08:00
Ken Huang 6eddaeba58
KAFKA-18532: Clean Partition.scala zookeeper logic (#18594)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2025-01-18 17:54:06 -08:00
TaiJuWu ff3de0cedc
MINOR: restore testUnauthorizedTopicMetadataRequest (#18578)
This was removed during removal of zk code (#18542), but
we should instead convert it to work with kraft.

Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-18 17:34:10 -08:00
PoAn Yang 85ec8b50c8
KAFKA-18423: Remove ZkData and related unused references (#18605)
Also removed:

* ZkNodeChangeNotificationListener
* KafkaZkClient

Reviewers: Ismael Juma <ismael@juma.me.uk>
2025-01-18 17:07:42 -08:00
TengYao Chi 029d9184c6
KAFKA-18565 Cleanup SaslSetup (#18586)
Reviewers: Christo Lolov <lolovc@amazon.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-19 03:14:59 +08:00
Ken Huang a814e21da8
KAFKA-18430 Remove ZkNodeChangeNotificationListener (#18606)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-19 01:34:30 +08:00
PoAn Yang e124d3975b
KAFKA-806: Index may not always observe log.index.interval.bytes (#18012)
Currently, each log.append() will add at most 1 index entry, even when the appended data is larger than log.index.interval.bytes. One potential issue is that if a follower restarts after being down for a long time, it may fetch data much bigger than log.index.interval.bytes at a time. This means that fewer index entries are created, which can increase the fetch time from the consumers.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>
2025-01-17 11:51:05 -08:00
Ken Huang a6faec179a
KAFKA-18515 Remove DelegationTokenManagerZk (#18595)
Reviewers: TengYao Chi <kitingiao@gmail.com>, Christo Lolov <lolovc@amazon.com>
2025-01-17 16:47:33 +00:00
Ismael Juma 3996e90ff7
Remove casts to KRaftMetadataCache (#18579)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2025-01-17 07:09:34 -08:00
Alyssa Huang 4583b033f0
KAFKA-17642: PreVote response handling and ProspectiveState (#18240)
This PR implements the second part of KIP-996 and KAFKA-16164 (tasks KAFKA-16607, KAFKA-17642, KAFKA-17643, KAFKA-17675) which encompass the response handling of PreVotes, addition of new ProspectiveState, update to metrics, and addition of Raft simulation tests.

Voters now transition to ProspectiveState first before CandidateState to prevent unnecessary epoch bumps. Voters in ProspectiveState send PreVotes requests which are Vote requests with PreVote set to true.

Follower grants PreVotes if it has not yet fetched successfully from leader. Leader denies all PreVotes. Unattached, Prospective, Candidate, and Resigned will grant PreVotes if the requesting replica's log is at least as long as theirs. Granted PreVotes are not persisted like standard votes. It is possible for a voter to grant several PreVotes in the same epoch.

The only state which is allowed to transition directly to CandidateState is ProspectiveState. This happens on reception of majority of granted PreVotes or if at least one voter doesn't support PreVote requests.

Prospective will transition to Follower after election loss/timeout if it was already aware of last known leader and the leader's endpoint, or at any point if it discovers the leader.

Prospective will transition to Unattached after election loss/timeout if it does not know the leader endpoints.

After electionTimeout, Resigned now always transitions to Unattached and increases the epoch.

Prospective grants standard votes if it has not already granted a standard vote (no votedKey), has no leaderId, and the recipient's log is current enough

Candidate no longer backs off after election timeout. Candidate still backs off after election loss.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2025-01-17 09:38:03 -05:00
TengYao Chi 3191fe56fc
KAFKA-18413: Remove AdminZkClient (#18585)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2025-01-17 15:10:31 +01:00
PoAn Yang 5a1fb1588d
KAFKA-18373: Remove ZkMetadataCache (#18553)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ismael Juma <ismael@juma.me.uk>
2025-01-17 11:49:11 +01:00
PoAn Yang 0e502e0b47
KAFKA-18431: Remove KafkaController (#18573)
Remove KafkaController and related unused references:

* ControllerChannelContext
* ControllerChannelManager
* ControllerEventManager
* ControllerState
* PartitionStateMachine
* ReplicaStateMachine
* TopicDeletionManager
* ZkBrokerEpochManager

Reviewers: Ismael Juma <ismael@juma.me.uk>
2025-01-16 18:56:05 -08:00
Kirk True 4f0a91393c
KAFKA-17623: Flaky testSeekPositionAndPauseNewlyAssignedPartitionOnPartitionsAssignedCallback (#18515)
Reviewers: Lianet Magrans <lmagrans@confluent.io>
2025-01-16 22:02:23 +01:00
Ken Huang cabbb613a2
KAFKA-18407: Remove ZkAdminManager, DelayedCreatePartitions, CreatePartitionsMetadata, ZkConfigRepository, DelayedDeleteTopics (#18574)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2025-01-16 19:32:22 +01:00
TengYao Chi 60cc2a0dea
KAFKA-18556: Remove JaasModule#zkDigestModule, JaasTestUtils#zkSections (#18568)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2025-01-16 18:17:06 +01:00
Mickael Maison 6eb44ad869
KAFKA-14485: Move LogCleaner exceptions to storage module (#18534)
Reviewers: Luke Chen <showuon@gmail.com>, Ken Huang <s7133700@gmail.com>
2025-01-16 17:26:29 +01:00
Jason Taylor 54fe0f0135
KAFKA-16368: Add a new constraint for segment.bytes to min 1MB for KIP-1030 (#18140)
Reviewers: Divij Vaidya <diviv@amazon.com>
2025-01-16 16:07:00 +01:00
TengYao Chi 724cf84de9
MINOR: Replace deprecated RichOptional.asScala with toScala (#18529)
Reference: https://www.scala-lang.org/api/2.13.0/scala/jdk/OptionConverters$$RichOptional.html

Reviewers: Ismael Juma <ismael@juma.me.uk>
2025-01-16 06:32:49 -08:00
Jason Taylor 23c459286b
KAFKA-16368: Update defaults for LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_DEFAULT and NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG (#18106)
Reviewers: Divij Vaidya <diviv@amazon.com>
2025-01-16 15:30:00 +01:00
Ken Huang ce1b079884
KAFKA-18542 Cleanup AlterPartitionManager (#18552)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-16 19:50:02 +08:00
Ken Huang 762bbcb711
KAFKA-18406 Remove ZkBrokerEpochManager.scala (#18561)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-16 19:35:09 +08:00
PoAn Yang 25f2ed090a
KAFKA-18405 Remove ZooKeeper logic from DynamicBrokerConfig (#18508)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-16 19:23:59 +08:00
Jason Taylor 11c10fe4da
KAFKA-16368: Update default linger.ms to 5ms for KIP-1030 (#18080)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Divij Vaidya <diviv@amazon.com>
2025-01-16 10:50:06 +01:00
PoAn Yang 14daa23b59
KAFKA-18331: Make process.roles and node.id required configs (#18414)
In 4.0, there is no ZK mode and both of these configs are required in kraft mode.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2025-01-15 23:55:51 -08:00
Chung, Ming-Yen 60d08a7abb
KAFKA-18552: Remove unnecessary version check in testHandleOffsetFetch* (#18559)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2025-01-15 19:39:19 -08:00
Ken Huang b9ccab42fe
KAFKA-18472: Remove MetadataSupport (#18483)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-15 19:38:33 -08:00
Dmitry Werner 92fd99bda1
KAFKA-18479: Remove keepPartitionMetadataFile in UnifiedLog and LogMan… (#18491)
Reviewers: Jun Rao <junrao@gmail.com>
2025-01-15 13:59:28 -08:00
Apoorv Mittal 3fa998475b
KAFKA-18539 Remove optional managers in KafkaApis (#18550)
Removed Optional for SharePartitionManager and ClientMetricsManager as zookeeper code is being removed. Also removed asScala and asJava conversion in KafkaApis.handleListClientMetricsResources, moved to java stream.

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-16 04:46:05 +08:00
Sushant Mahajan 47f22faac3
MINOR: Added flaky references for a few tests. (#18558)
Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-01-15 19:24:52 +00:00
Apoorv Mittal 06400177bb
KAFKA-18452: Implemented batch size in acquired records (#18459)
The PR implements a way to divide acquired batches into batch size as desirable by client.

The BatchSize is the soft limit and should align the batches in response and cached state in broker at the log batch boundaries.

Reviewers:  Andrew Schofield <aschofield@confluent.io>, Abhinav Dixit <adixit@confluent.io>, Jun Rao <junrao@gmail.com>
2025-01-15 11:08:05 -08:00
Kuan-Po Tseng d3b4c1bdf4
KAFKA-18401: Transaction version 2 does not support commit transaction without records (#18448)
Fix the issue where producer.commitTransaction under transaction version 2 throws error if no partition or offset is added to transaction. The solution is to avoid sending the endTxnRequest unless producer.send or producer.sendOffsetsToTransaction is triggered.

Reviewers: Justine Olshan <jolshan@confluent.io>
2025-01-15 10:21:11 -08:00
Apoorv Mittal e12db663f0
KAFKA-18514 Remove server dependency on share coordinator (#18536)
The PR removes dependency of server module on share-coordinator, rather it should be other way. Moved the ShareCoordinatorConfig class from server to share-coordinator.

Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-16 00:47:01 +08:00
Sushant Mahajan e3a56f3162
KAFKA-18513: Validate share state topic records produced in tests. (#18521)
Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-01-15 16:10:07 +00:00
TaiJuWu d96b68252c
KAFKA-18399 Remove ZooKeeper from KafkaApis (12/N): clean up ZKMetadataCache, KafkaController and raftSupport (#18542)
Reviewers: Viktor Somogyi-Vass <viktorsomogyi@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-15 23:28:57 +08:00
Ted Yan 5b8319e6c2
KAFKA-18399 Remove ZooKeeper from KafkaApis (11/N): CREATE_ACLS and DELETE_ACLS (#18540)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-15 18:46:50 +08:00
TaiJuWu 3fcaec7f4e
KAFKA-18399 Remove ZooKeeper from KafkaApis (10/N): ALTER_CONFIG and INCREMENETAL_ALTER_CONFIG (#18432)
Reviewers: Christo Lolov <lolovc@amazon.com>, Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-15 18:25:45 +08:00
PoAn Yang ae661dec34
KAFAK-18451: Flaky RemoteLogManagerTest#testRLMOpsWhenMetadataIsNotReady (#18520)
The REMOTE_LOG_MANAGER_TASK_INTERVAL_MS_PROP in RemoteLogManagerTest is 100 which is too small. If assertions verifyNoMoreInteractions can't run in 100ms, the scheduler will run RLMTask again and the case will fail.

Reviewers: Luke Chen <showuon@gmail.com>
2025-01-15 16:18:06 +08:00
Ismael Juma f3a93551fa
Revert "KAFKA-18034: CommitRequestManager should fail pending requests on fatal coordinator errors (#18050)" (#18544)
This reverts commit 70d6312a3a.

Reviewers: Luke Chen <showuon@gmail.com>
2025-01-15 16:16:47 +08:00
TengYao Chi 118e1835cc
KAFKA-18502 Remove kafka.controller.Election (#18518)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-15 05:42:16 +08:00
mingdaoy 9f955973fe
KAFKA-18399 Remove ZooKeeper from KafkaApis (9/N): ALTER_CLIENT_QUOTAS and ALLOCATE_PRODUCER_IDS (#18465)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-15 05:06:16 +08:00
TaiJuWu ddc3faa88f
KAFKA-18399 Remove ZooKeeper from KafkaApis (8/N): ELECT_LEADERS , ALTER_PARTITION, UPDATE_FEATURES (#18453)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-15 04:53:03 +08:00
Ken Huang c00c576b65
MINOR: Fix some comments typo (#18509)
Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-01-14 15:00:02 +00:00
Sanskar Jhajharia e3e4c17959
Add DescribeShareGroupOffsets API [KIP-932] (#18500)
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2025-01-14 14:33:39 +00:00
David Jacot f6912a9a6a
MINOR: Fix typo in DumpLogSegments' TransactionLogMessageParser (#18505)
The value should use `version` instead of `type`.

Reviewers: Jeff Kim <jeff.kim@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-14 08:10:10 +01:00
Peter Lee da0c3beffa
KAFKA-18491 Remove zkClient & maybeUpdateMetadataCache from ReplicaManager (#18507)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-14 03:20:23 +08:00
Ken Huang 70d6312a3a
KAFKA-18034: CommitRequestManager should fail pending requests on fatal coordinator errors (#18050)
Reviewers: Kirk True <ktrue@confluent.io>, Lianet Magrans <lmagrans@confluent.io>
2025-01-13 15:29:14 +01:00
Abhinav Dixit 4e24c50ba3
KAFKA-18404: Remove partitionMaxBytes usage from DelayedShareFetch (#17870)
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2025-01-13 13:56:19 +00:00
TengYao Chi 5d95d91807
MINOR: Javadoc fixes in KRaftMetadataCache (#18493)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2025-01-13 14:37:03 +01:00
TaiJuWu 9f5d9f3cd4
KAFKA-18399 Remove ZooKeeper from KafkaApis (7/N): CREATE_TOPICS, DELETE_TOPICS, CREATE_PARTITIONS (#18433)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-13 21:23:59 +08:00
Ken Huang 0b60d08d23
KAFKA-18341: Remove KafkaConfig GroupType config check and warn log (#18320)
As ZK mode is being removed for 4.0, we don't need this check anymore.

Reviewers: David Jacot <djacot@confluent.io>
2025-01-13 02:42:17 -08:00
David Jacot 273719227e
KAFKA-18457; Update DumpLogSegments to use coordinator record json converters (#18480)
This patch updates the ShareGroupStateMessageParser and OffsetsMessageParser used by the DumpLogSegments command line tool to use the recently introduced json converters for those records. It basically means that new records are automatically supported.

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-13 11:28:54 +01:00
Ken Huang 33556aedc3
KAFKA-18399 Remove ZooKeeper from KafkaApis (6/N): `handleCreateTokenRequest`, `handleRenewTokenRequestZk`, `handleExpireTokenRequestZk` (#18447)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-13 01:42:32 +08:00
Ken Huang 6a8ffe7f64
KAFKA-18399 Remove ZooKeeper from KafkaApis (5/N): ALTER_PARTITION_REASSIGNMENTS, LIST_PARTITION_REASSIGNMENTS (#18464)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-13 01:33:24 +08:00
PoAn Yang 3cf2e45dc4
KAFKA-18399 Remove ZooKeeper from KafkaApis (4/N): OFFSET_COMMIT and OFFSET_FETCH (#18461)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-12 20:54:10 +08:00
TengYao Chi 81b1508acb
KAFKA-18399 Remove ZooKeeper from KafkaApis (3/N): USER_SCRAM_CREDENTIALS (#18456)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-12 20:39:49 +08:00
Xuan-Zhang Gong 1ee715473c
KAFKA-18446 Remove MetadataCacheControllerNodeProvider (#18437)
Reviewers: PoAn Yang <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-12 02:12:35 +08:00
Ismael Juma d4aee71e36
KAFKA-18465: Remove MetadataVersions older than 3.0-IV1 (#18468)
Apache Kafka 4.0 will only support KRaft and 3.0-IV1 is the minimum version supported by KRaft. So, we can assume that Apache Kafka 4.0 will only communicate with brokers that are 3.0-IV1 or newer.

Note that KRaft was only marked as production-ready in 3.3, so we could go further and set the baseline to 3.3. I think we should have that discussion, but it made sense to start with the non controversial parts.

Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <david.jacot@gmail.com>
2025-01-11 09:42:39 -08:00
Justine Olshan 32dbbe6a1f
KAFKA-18464: Empty Abort Transaction can fence producer incorrectly with Transactions V2 (#18467)
To avoid self-fencing in the commit/abort + empty abort scenario, return the concurrent transactions error when we have pending state and do the epoch check second. In this scenario, we will complete the previous transaction before proceeding to the empty abort.

Added a test that failed before the change.

Note -- only the pending state is checked earlier. This is because we don’t return from EndTxn (the first commit) until we already written to the log, transitioned to PrepareX, and have the pending CompleteX state. We don't need to worry about the cases of an EndTxn request coming in with PrepareX without the pending state because that would be an older request and/or retry which are already covered.

Reviewers: Artem Livshits <alivshits@confluent.io>, Jeff Kim <jeff.kim@confluent.io>
2025-01-10 16:51:58 -08:00
PoAn Yang a7342a2e62
MINOR: fix flaky RemoteLogManagerTest#testStopPartitionsWithDeletion (#18474)
The test has become flakier recently and it's easy to reproduce by running the single test (vs
running the the class test suite).

The root cause is that following functions call `RemoteLogMetadataManager#listRemoteLogSegments`.
It returns iterator. If one of function goes through iterator first, another can't get expected result.
I changed `thenReturn` to `thenAnswer` to avoid the issue.

The race is between:
* RLMExpirationTask#cleanupExpiredRemoteLogSegments
* RemoteLogManager#deleteRemoteLogPartition

Reviewers: Ismael Juma <ismael@juma.me.uk>

Signed-off-by: PoAn Yang <payang@apache.org>
2025-01-10 06:33:10 -08:00
TaiJuWu 5684fc7a2e
KAFKA-18399 Remove ZooKeeper from KafkaApis (2/N): CONTROLLED_SHUTDOWN and ENVELOPE (#18422)
Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-10 04:12:12 +08:00
Ismael Juma cf7029c026
KAFKA-13093: Log compaction should write new segments with record version v2 (KIP-724) (#18321)
Convert v0/v1 record batches to v2 during compaction even if said record batches would be
written with no change otherwise. A few important details:

1. V0 compressed record batch with multiple records is converted into single V2 record batch
2. V0 uncompressed records are converted into single record V2 record batches
3. V0 records are converted to V2 records with timestampType set to `CreateTime` and the
timestamp is `-1`.
4. The `KAFKA-4298` workaround is no longer needed since the conversion to V2 fixes
the issue too.
5. Removed a log warning applicable to consumers older than 0.10.1 - they are no longer
supported.
6. Added back the ability to append records with v0/v1 (for testing only).
7. The creation of the leader epoch cache is no longer optional since the record version
config is effectively always V2.

Add integration tests, these tests existed before #18267 - restored, modified and
extended them.

Reviewers: Jun Rao <jun@confluent.io>
2025-01-09 09:37:23 -08:00
Peter Lee a116753cc8
KAFKA-17986 Fix ConsumerRebootstrapTest and ProducerRebootstrapTest (#18175)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-10 01:02:34 +08:00
Ken Huang 307059c770
KAFKA-18399 Remove ZooKeeper from KafkaApis (1/N): `LEADER_AND_ISR`, `STOP_REPLICA`, `UPDATE_METADATA` (#18417)
Delete the handlers for LEADER_AND_ISR, STOP_REPLICA, and UPDATE_METADATA. Also, remove the corresponding unit tests in KafkaApisTest.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-10 00:01:38 +08:00
TengYao Chi 6fa5537ceb
MINOR: Replace ImplicitConversions with CollectionConverters (#18412)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2025-01-09 10:00:55 +01:00
Colin Patrick McCabe c28d9a3486
KAFKA-18435 Remove zookeeper dependencies in build.gradle (#18450)
Remove Apache ZooKeeper from the Apache Kafka build. Also remove commons IO, commons CLI, and netty, which were dependencies we took only because of ZooKeeper.

In order to keep the size of this PR manageable, I did not remove all classes which formerly interfaced with ZK. I just removed the ZK types. Fortunately, Kafka generally wrapped ZK data structures rather than using them directly.

Some classes were pretty entangled with ZK, so it was easier just to stub them out. For ZkNodeChangeNotificationListener.scala, PartitionStateMachine.scala, ReplicaStateMachine.scala, KafkaZkClient.scala, and ZookeeperClient.scala, I replaced all the functions with "throw new UnsupportedOperationException". Since the tests for these classes have been removed, as well as the ZK-based broker code, this should be OK as an incremental step.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-09 10:25:17 +08:00
Logan Zhu 5efaae65c6
KAFKA-18432 Remove unused code from AutoTopicCreationManager (#18438)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-09 02:52:28 +08:00
PoAn Yang 4b1b67e3c4
KAFKA-18434: enrich the authorization error message of connecting to controller (#18436)
Reviewers: Divij Vaidya <diviv@amazon.com>
2025-01-08 18:56:29 +01:00
TengYao Chi af3f9e3a5a
KAFKA-18426 Remove FinalizedFeatureChangeListener (#18441)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-09 01:24:03 +08:00
Justine Olshan f79d7dc3f4
KAFKA-18035: Readd flaky annotation back for testBumpTransactionalEpochWithTV2Disabled (#18426)
Reviewers: David Jacot <djacot@confluent.io>
2025-01-08 09:06:41 -08:00
TengYao Chi aa22676c48
KAFKA-18425 Remove OffsetTrackingListener (#18443)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-09 00:18:49 +08:00
Andrew Schofield 3f9d2c2db0
KAFKA-18433: Add BatchSize to ShareFetch request (1/N) (#18439)
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
2025-01-08 15:29:43 +00:00
Peter Lee 0377e807ff
MINOR: Use Producer interface and ClusterInstance producer factory (#18197)
Reviewers: David Arthur <mumrah@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-08 20:53:30 +08:00
Peter Lee 08ef22d888
KAFKA-18173 Remove duplicate `assertFutureError` (#18296)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-08 20:24:35 +08:00
Mickael Maison d1aa370db8
KAFKA-15599: Move SegmentPosition/TimingWheelExpirationService to raft module (#18094)
Reviewers: Divij Vaidya <diviv@amazon.com>, Jason Taylor <jastaylr@amazon.com>
2025-01-08 12:49:38 +01:00
TaiJuWu 0c435e3855
KAFKA-18353 Remove zk config `control.plane.listener.name` (#18329)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-08 18:38:37 +08:00
TengYao Chi 8568157f7f
KAFKA-18443 Remove ZkFourLetterWords (#18429)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-08 17:24:36 +08:00
Jhen-Yung Hsu f95726a211
KAFKA-18417 Remove controlled.shutdown.max.retries and controlled.shutdown.retry.backoff.ms (#18431)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-08 17:13:42 +08:00
Ken Huang 6aef94e9ec
KAFKA-18411 Remove ZkProducerIdManager (#18413)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-08 04:23:27 +08:00
TengYao Chi 2a073a14d2
KAFKA-18414 Remove KRaftRegistrationResult (#18401)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-08 04:09:28 +08:00
Ken Huang 9d93a4f68f
KAFKA-18384 Remove ZkAlterPartitionManager (#18364)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2025-01-07 18:08:57 +01:00
TengYao Chi af255a0c37
KAFKA-18412: Remove EmbeddedZookeeper (#18399)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2025-01-07 17:08:00 +01:00
NICOLAS GUYOMAR 2fc35c81be
MINOR : Improve Exception log in NotEnoughReplicasException(#12394)
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2025-01-07 20:21:02 +05:30
David Jacot 48b522fe86
MINOR: Improve PlaintextAdminIntegrationTest#testConsumerGroups (#18409)
Looking at the [history](https://ge.apache.org/scans/tests?search.rootProjectNames=kafka&search.timeZoneId=Europe%2FZurich&tests.container=kafka.api.PlaintextAdminIntegrationTest&tests.test=testConsumerGroups(String%2C%20String)%5B2%5D), I found out that one source of flakiness is due to syncCommit failing with CommitFailedException. We can ignore it and retry on the next iteration.

```
[2025-01-07 10:17:00,783] ERROR [Consumer instanceId=test_instance_id_1, clientId=test_client_id, groupId=test_group_id] OffsetCommit failed for member VfImExrxT3-w_HNJcTkqnw with stale member epoch error. Last epoch sent: 2 (org.apache.kafka.clients.consumer.internals.CommitRequestManager:773)
Exception in thread "Thread-6" org.apache.kafka.clients.consumer.CommitFailedException: OffsetCommit failed with stale member epoch.The member epoch is stale. The member must retry after receiving its updated member epoch via the ConsumerGroupHeartbeat API.
        at org.apache.kafka.clients.consumer.internals.CommitRequestManager.commitSyncExceptionForError(CommitRequestManager.java:481)
        at org.apache.kafka.clients.consumer.internals.CommitRequestManager.lambda$commitSyncWithRetries$7(CommitRequestManager.java:472)
        at java.base/java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:863)
        at java.base/java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:841)
        at java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:510)
        at java.base/java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:2162)
        at org.apache.kafka.clients.consumer.internals.CommitRequestManager$OffsetCommitRequestState.onResponse(CommitRequestManager.java:776)
        at org.apache.kafka.clients.consumer.internals.CommitRequestManager$RetriableRequestState.handleClientResponse(CommitRequestManager.java:893)
        at org.apache.kafka.clients.consumer.internals.CommitRequestManager$RetriableRequestState.lambda$buildRequestWithResponseHandling$0(CommitRequestManager.java:883)
        at java.base/java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:863)
        at java.base/java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:841)
        at java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:510)
        at java.base/java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:2147)
        at org.apache.kafka.clients.consumer.internals.NetworkClientDelegate$FutureCompletionHandler.onComplete(NetworkClientDelegate.java:433)
        at org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:154)
        at org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:669)
        at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:661)
        at org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.poll(NetworkClientDelegate.java:153)
        at org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread.runOnce(ConsumerNetworkThread.java:160)
        at org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread.run(ConsumerNetworkThread.java:106)
```

Reviewers: Lianet Magrans <lmagrans@confluent.io>
2025-01-07 06:27:26 -08:00
Colin Patrick McCabe d8236bec44
MINOR: Remove RaftManager.maybeDeleteMetadataLogDir and AutoTopicCreationManagerTest.scala (#17365)
Remove RaftManager.maybeDeleteMetadataLogDir since it was only used during ZK migration, and that code has been removed.

Similarly, remove RaftManagerTest.testKRaftBrokerDoesNotDeleteMetadataLog which tested that function.

Remove AutoTopicCreationManagerTest since it tests the ZK-mode-only AutoTopicReationManager.

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-07 21:18:58 +08:00
Ken Huang d874aa42f3
KAFKA-18368 Remove TestUtils#MockZkConnect and remove zkConnect from TestUtils#createBrokerConfig (#18352)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-07 21:03:13 +08:00
David Jacot 7b6e94642a
KAFKA-18303; Update ShareCoordinator to use new record format (#18396)
Following https://github.com/apache/kafka/pull/18261, this patch updates the Share Coordinator to use the new record format.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2025-01-06 23:59:07 -08:00
David Arthur c4840f5e93
KAFKA-16446: Improve controller event duration logging (#15622)
There are times when the controller has a high event processing time, such as during startup, or when creating a topic with many partitions. We can see these processing times in the p99 metric (kafka.controller:type=ControllerEventManager,name=EventQueueProcessingTimeMs), however it's difficult to see exactly which event is causing high processing time.

With DEBUG logs, we see every event along with its processing time. Even with this, it's a bit tedious to find the event with a high processing time.

This PR logs all events which take longer than 2 seconds at ERROR level. This will help identify events that are taking far too long, and which could be disruptive to the operation of the controller. The slow event logging looks like this:

```
[2024-12-20 15:03:39,754] ERROR [QuorumController id=1] Exceptionally slow controller event createTopics took 5240 ms.  (org.apache.kafka.controller.EventPerformanceMonitor)
```

Also, every 60 seconds, it logs some event time statistics, including average time, maximum time, and the name of the event which took the longest. This periodic message looks like this:

```
[2024-12-20 15:35:04,798] INFO [QuorumController id=1] In the last 60000 ms period, 333 events were completed, which took an average of 12.34 ms each. The slowest event was handleCommit[baseOffset=0], which took 41.90 ms. (org.apache.kafka.controller.EventPerformanceMonitor)
```

An operator can disable these logs by adding the following to their log4j config:

```
org.apache.kafka.controller.EventPerformanceMonitor=OFF
```

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2025-01-06 13:34:46 -08:00
Zhihong Yu c116354908
MINOR: Use clear method to speed up removal (#18400)
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2025-01-06 17:50:18 +00:00
Mickael Maison 64bbdb1a03
KAFKA-17616: Remove KafkaServer (#18384)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Ken Huang <s7133700@gmail.com>
2025-01-06 14:36:08 +01:00
Ken Huang a628d9bc4d
KAFKA-18365 Remove zookeeper.connect in Test (#18353)
Reviewers: TaiJuWu <tjwu1217@gmail.com>, PoAn Yang <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-04 23:04:21 +08:00
Ismael Juma 73ab7ee4ea
MINOR: Use `Files.readString/writeString` and `String.repeat` to simplify code (#18372)
The 3 methods were introduced in Java 11.

Reviewers: Divij Vaidya <diviv@amazon.com>
2025-01-02 17:50:27 -08:00
David Arthur a2a8d87153
MINOR remove some flaky annotations (#18357)
Remove the flaky annotation from the following tests

* RemoteLogManagerTest#testFetchOffsetByTimestampWithTieredStorageDoesNotFetchIndexWhenExistsLocally
* All the children of BaseConsumerTest#testCoordinatorFailover
* TransactionsTest#testFailureToFenceEpoch
* TransactionsTest#testReadCommittedConsumerShouldNotSeeUndecidedData
* MetricsDuringTopicCreationDeletionTest#testMetricsDuringTopicCreateDelete
* ProduceRequestTest#testProduceWithInvalidTimestamp

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-02 11:04:50 -05:00
xijiu dbf4602c13
KAFKA-18367 Remove ZkConfigManager (#18363)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-01 21:16:25 +08:00
xijiu 5f8cf0e1f5
KAFKA-17421 Add integration tests for ConsumerRecord#leaderEpoch (#18254)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-01 00:37:31 +08:00
TengYao Chi 3161115ada
KAFKA-18361 Remove PasswordEncoderConfigs (#18347)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-31 00:18:23 +08:00
Mickael Maison 4080f19c5c
KAFKA-17615 Remove KafkaServer from tests (#18271)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-30 22:57:16 +08:00
TaiJuWu 03f16f368b
KAFKA-18313 Fix to Kraft or remove tests associate with Zk Broker config in SocketServerTest and ReplicaFetcherThreadTest (#18327)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-30 02:27:40 +08:00
Ken Huang f2ae20a5c1
KAFKA-18316 Fix to Kraft or remove tests associate with Zk Broker config in ConnectionQuotasTest (#18279)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-30 02:20:10 +08:00
Ismael Juma 3654bc4513
KAFKA-18339: Fix parseRequestHeader error handling (#18340)
A minor refactoring just before merging #18295 introduced a regression and no test failed. Throw the correct exception and add test to verify it. Also refactor the code slightly to make that possible.

Thanks to Chia-Ping for catching the issue.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-29 09:31:14 -08:00
Ken Huang 1156d5c294
KAFKA-18315 Fix to Kraft or remove tests associate with Zk Broker config in DynamicBrokerConfigTest, ReplicaManagerTest, DescribeTopicPartitionsRequestHandlerTest, KafkaConfigTest (#18269)
Reviewers: PoAn Yang <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-30 00:47:46 +08:00
TengYao Chi e9a03b3a86
KAFKA-18314 Fix to Kraft or remove tests associate with Zk Broker config in KafkaApisTest (#18309)
Reviewers: TaiJuWu <tjwu1217@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-29 19:02:27 +08:00
PoAn Yang 91a2b58616
KAFKA-18359 Set zkConnect to null in LocalLeaderEndPointTest, HighwatermarkPersistenceTest, IsrExpirationTest, ReplicaManagerQuotasTest, OffsetsForLeaderEpochTest (#18344)
Reviewers: TaiJuWu <tjwu1217@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-29 18:23:42 +08:00
Ken Huang bc7a1a8969
KAFKA-18317 Remove zookeeper.connect from RemoteLogManagerTest (#18283)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-28 21:51:08 +08:00
Ismael Juma 875da35ec3
KAFKA-18339: Remove raw unversioned direct SASL protocol (KIP-896) (#18295)
Clients that support SASL but don't implement KIP-43 (eg Kafka producer/consumer 0.9.0.x) will
fail to connect after this change.

Added unit tests and also manually tested with the console producer 0.9.0.

While testing, I noticed that the logged message when a 0.9.0 Java client is used without sasl is
slightly misleading - fixed that too.

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2024-12-27 10:23:25 -08:00
Ismael Juma dfb178a1d8
KAFKA-18272: Deprecated protocol api usage should be logged at info level (#18313)
This makes it possible to enable request logs for deprecated protocol api versions without enabling it for the rest. Combined with the ability to enable/disable dynamically, it makes it a bit easier to collect the information about deprecated clients that is not available via metrics.

This isn't particularly useful in trunk/4.0 since there are no deprecated api versions in these versions, but it will be useful for older branches. I intend to backport to those branches and add a release note in the backport regarding the change in behavior.

I manually verified that:
1. If the request logger is configured at `INFO` level, only deprecated protocol api versions are logged and they are logged at `INFO` level.
2. If the request logger is configured at `DEBUG` level, all requests are logged but the log level is `INFO` for deprecated protocol api versions and `DEBUG` for the rest.
3. If the request logger is configured at `WARN` level (the default), no requests are logged.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-27 00:04:19 -08:00
Peter Lee c7c1364b0f
KAFKA-18274 Failed to restart controller in testing due to closed socket channel [1/2] (#18310)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-25 20:52:06 +08:00
PoAn Yang b4be178599
KAFKA-17393: Remove log.message.format.version/message.format.version (KIP-724) (#18267)
Based on [KIP-724](https://cwiki.apache.org/confluence/display/KAFKA/KIP-724%3A+Drop+support+for+message+formats+v0+and+v1), the `log.message.format.version` and `message.format.version` can be removed in 4.0.

These configs effectively a no-op with inter-broker protocol version 3.0 or higher
since Apache Kafka 3.0, so the impact should be minimal.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2024-12-21 15:35:15 -08:00
Justine Olshan 8bd3746e0c
KAFKA-17705: Add Transactions V2 system tests and mark as production ready (#18132)
Added transaction version 2 to some of the system tests. Also marking TV2 as production ready.

Also fixes the defaultVersion test. 

Reviewers: Jun Rao <jun@confluent.io>
2024-12-21 14:01:54 -08:00
Ismael Juma fe56fc98fa
KAFKA-18269: Remove deprecated protocol APIs support (KIP-896, KIP-724) (#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 <mumrah@gmail.com>
2024-12-20 19:52:00 -08:00
Ismael Juma 288d4de834
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>
2024-12-20 16:59:51 -08:00
Artem Livshits 3ef10a3f61
KAFKA-15370: Support Participation in 2PC (KIP-939) (1/N) (#17687)
This is just a mechanical change to make prepareTransitionTo method use named parameters instead of positional parameters.

Reviewers: Justine Olshan <jolshan@confluent.io>, Ritika Reddy <rreddy@confluent.io>
2024-12-20 14:09:32 -08:00
PoAn Yang 748ad6a793
HOTFIX: fix incompatible types: Optional<TimestampAndOffset> cannot be converted to Option<TimestampAndOffset> (#18284)
Signed-off-by: PoAn Yang <payang@apache.org>

Reviewers: Christo Lolov <lolovc@amazon.com>
2024-12-20 14:57:33 +00:00
Ken Huang e8863c9ee2
KAFKA-18180: Move OffsetResultHolder to storage module (#18100)
Reviewers: Christo Lolov <lolovc@amazon.com>
2024-12-20 11:52:34 +00:00
David Jacot d67379c310
KAFKA-18301; Make coordinator records first class citizen (#18261)
This patch is the first one in a series to improve how coordinator records are managed. It focuses on making coordinator records first class citizen in the generator.
* Introduce `coordinator-key` and `coordinator-value` in the schema;
* Introduce `apiKey` for those. This is done to avoid relying on the version to determine the type.
* It also allows the generator to enforce some rules: the key cannot use flexible versions, the key must have a single version `0`, there must be a key and a value for a given api key, etc.
* It generates an enum with all the coordinator record types. This is pretty handy in the code.

The patch also updates the group coordinators to use those.

Reviewers: Jeff Kim <jeff.kim@confluent.io>, Andrew Schofield <aschofield@confluent.io>
2024-12-20 12:16:14 +01:00
Justine Olshan e099fce567
KAFKA-18035, KAFKA-18306, KAFKA-18092: Address TransactionsTest flaky tests (#18264)
A lot of these tests assumed that the commit/abort happened immediately. Spoiler alert -- it does not.

For some I ensure that the first send of the next transaction is successful before grabbing the epoch. I also loosened some checks since we don't need to guarantee the exact epoch.

I went back and forth with completely deleting testBumpTransactionalEpochWithTV2Enabled since we don't have client side epoch bumps with V2 (which is what the test was originally testing), but I opted to keep it to just confirm the epoch on each transaction -- even in the timeout scenario.

Reviewers: Calvin Liu <caliu@confluent.io>, Artem Livshits <alivshits@confluent.io>, Jeff Kim <jeff.kim@confluent.io>, David Mao <dmao@confluent.io>
2024-12-19 14:06:37 -08:00
kevin-wu24 9f67ee3a9d
KAFKA-18305: validate controller.listener.names is not in inter.broker.listener.name for kcontrollers (#18222)
When inter.broker.listener is explicitly set, validate that it is not in the set of controller.listener.names.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, David Arthur <mumrah@gmail.com>
2024-12-19 13:29:38 -08:00
Mickael Maison 3fafa096b1
KAFKA-18207: Serde for handling transaction records (#18136)
Reviewers: Andrew Schofield <aschofield@confluent.io>
2024-12-19 21:39:09 +01:00
David Arthur 64279d2e82
Mark flaky tests for Dec 18, 2024 (#18263)
Reviewers: TengYao Chi <kitingiao@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2024-12-19 10:12:33 -05:00
Peter Lee 3bc366f1a4
Add IT for share consumer with duration base offet auto reset (#18251)
Reviewers: Andrew Schofield <aschofield@confluent.io>
2024-12-18 21:47:38 +00:00
Lucas Brutschy 0055ef0a49
KAFKA-18283: Add StreamsGroupDescribe RPC definitions (#18230)
Adds a new RPC StreamsGroupDescribe that returns, given the group ID, all metadata related to the streams group, such as

 - The topology metadata of the group.
 - The topology epoch of the group.
 - The latest member metadata that each member provided through the StreamsGroupHeartbeat API.
 - The current target assignment generated by the assignor.
 - This just adds the JSON as defined in KIP-1071, together with some plumbing.

Reviewers: Bill Bejeck <bbejeck@gmail.com>
2024-12-18 19:38:01 +01:00
Nick Guo 21b7bb2265
KAFKA-18264 Remove NotLeaderForPartitionException (#18211)
Reviewers: Yung <yungyung7654321@gmail.com>, Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-19 00:56:13 +08:00
Lucas Brutschy ec32c8a376
KAFKA-18282: Add StreamsGroupHeartbeat RPC definitions (#18227)
The StreamsGroupHeartbeat API is the new core API used by streams application to form a group. The API allows members to initialize a topology, advertise their state, and their owned tasks. The group coordinator uses it to assign/revoke tasks to/from members. This API is also used as a liveness check.

This change adds the JSON definition of the RPC, as defined in KIP-1071.

Reviewers: Bruno Cadonna <cadonna@apache.org>
2024-12-18 11:43:44 +01:00
PoAn Yang 0bf0033d96
MINOR: add assertion about groupEpoch and targetAssignmentEpoch to testConsumerGroups (#18203)
Reviewers: David Jacot <djacot@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-18 08:05:09 +08:00
David Jacot dfc07e0e0c
KAFKA-17960; PlaintextAdminIntegrationTest.testConsumerGroups fails with CONSUMER group protocol (#18234)
The issue has been fixed by https://issues.apache.org/jira/browse/KAFKA-18188. We can re-enable the test with the CONSUMER protocol.

Reviewers: Lianet Magrans <lmagrans@confluent.io>, Andrew Schofield <aschofield@confluent.io>
2024-12-17 20:09:45 +01:00
ShivsundarR e88fede27e
KAFKA-18025 Remove flaky tag from several tests in ShareConsumerTest (#18226)
Develocity report: https://ge.apache.org/scans/tests?search.relativeStartTime=P28D&search.rootProjectNames=kafka&search.tags=github,trunk&search.timeZoneId=America%2FNew_York&tests.container=kafka.test.api.ShareConsumerTest#

Reviewers: David Arthur <mumrah@gmail.com>
2024-12-17 12:46:21 -05:00
David Jacot a12152f028
KAFKA-18263; Group lock must be acquired when reverting static membership rejoin (#18207)
When a static member rejoins the group, the group state is rewritten to the partition in order to persist the change. If the write fails, the change is reverted. However, this is done without acquiring the group lock.

This is only try in the old group coordinator. The new one does not suffer from this issue.

Reviewers: Jeff Kim <jeff.kim@confluent.io>
2024-12-16 18:20:35 +01:00
Ken Huang 92f61b36f4
KAFKA-18226: Disable CustomQuotaCallbackTest and remove isKRaftTest (#18166)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-12-16 16:46:39 +01:00
Peter Lee 220c578521
KAFKA-18014: Add duration based offset reset option for ShareConsumer (#18096)
Kafka consumer supports auto.offset.reset config option, which is used when there is no initial offset in Kafka (or) if the current offset does not exist any more on the server. This config currently supports earliest/latest/none options. Currently consumer resets might force applications to reprocess large amounts of data from earlier offsets. With infinite storage, its beneficial to have a duration based offset reset strategy. This will allow applications to consume/initialise from a fixed duration when there is no initial offset in Kafka.

As part of KIP-932, we are adding support for share consumer groups. Share consumer groups supports dynamic group configuration property share.auto.offset.reset. This is used to set the initial Share-Partition Start Offset (SPSO) based on the share.auto.offset.reset configuration. Currently share.auto.offset.reset supports earliest and latest options to automatically reset the offset

Similar to the Kafka Consumer, we will add support for by_duration: config value for share.auto.offset.reset.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>
2024-12-16 08:25:37 +00:00
TengYao Chi 408ce19c62
KAFKA-18245 add Flaky annotation to testMetricsDuringTopicCreateDelete (#18187)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-15 15:48:11 +08:00
Kuan-Po Tseng 0815d70592
KAFKA-18160 Interrupting or waking up onPartitionsAssigned in AsyncConsumer can cause the ConsumerRebalanceListenerCallbackCompletedEvent to be skipped (#18089)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-15 10:47:30 +08:00
Kuan-Po Tseng fef625cfa2
KAFKA-18234 DumpLogSegments cannot print ConsumerGroupRegularExpression record (#18173)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-15 06:02:02 +08:00
Mickael Maison 57eb5fd7dc
KAFKA-14587: Move AclCommand to tools (#17880)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-14 20:05:46 +01:00
Kamal Chandraprakash 139e5b15a1
KAFKA-17928: Make remote log manager thread-pool configs dynamic (#17859)
- Disallow configuring -1 for copier and expiration thread pools dynamically

Co-authored-by: Peter Lee <peterxcli@gmail.com>

Reviewers: Peter Lee <peterxcli@gmail.com>, Satish Duggana <satishd@apache.org>
2024-12-14 13:14:05 +05:30
Kuan-Po Tseng 9e60fcc87f
KAFKA-18181 Refactor ShareConsumerTest (#18105)
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-14 09:39:44 +08:00
David Jacot 450c10d00c
KAFKA-17507; WriteTxnMarkers API must not return until markers are written and materialized in group coordinator's cache (#18168)
We have observed the below errors in some cluster:

Uncaught exception in scheduled task 'handleTxnCompletion-902667' exception.message:Trying to complete a transactional offset commit for producerId *** and groupId *** even though the offset commit record itself hasn't been appended to the log.

When a transaction is completed, the transaction coordinator sends a WriteTxnMarkers request to all the partitions involved in the transaction to write the markers to them. When the broker receives it, it writes the markers and if markers are written to the __consumer_offsets partitions, it informs the group coordinator that it can materialize the pending transactional offsets in its main cache. The group coordinator does this asynchronously since Apache Kafka 2.0, see this patch.

The above error appends when the asynchronous operation is executed by the scheduler and the operation finds that there are pending transactional offsets that were not written yet. How come?

There is actually an issue is the steps described above. The group coordinator does not wait until the asynchronous operation completes to return to the api layer. Hence the WriteTxnMarkers response may be send back to the transaction coordinator before the async operation is actually completed. Hence it is possible that the next transactional produce to be started also before the operation is completed too. This could explain why the group coordinator has pending transactional offsets that are not written yet.

There is a similar issue when the transaction is aborted. However on this path, we don't have any checks to verify whether all the pending transactional offsets have been written or not so we don't see any errors in our logs. Due to the same race condition, it is possible to actually remove the wrong pending transactional offsets.

PS: The new group coordinator is not impacted by this bug.

Reviewers: Justine Olshan <jolshan@confluent.io>
2024-12-13 13:50:41 -08:00
Ken Huang 669d8610a2
KAFKA-18228 The MetricsDuringTopicCreationDeletionTest should delete topics to ensure that the metrics are recreated (#18163)
Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-14 01:35:08 +08:00
TengYao Chi b37b89c668
KAFKA-9366 Upgrade log4j to log4j2 (#17373)
This pull request replaces Log4j with Log4j2 across the entire project, including dependencies, configurations, and code. The notable changes are listed below:

1. Introduce Log4j2 Instead of Log4j
2. Change Configuration File Format from Properties to YAML
3. Adds warnings to notify users if they are still using Log4j properties, encouraging them to transition to Log4j2 configurations

Co-authored-by: Lee Dongjin <dongjin@apache.org>

Reviewers: Luke Chen <showuon@gmail.com>, Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-14 01:14:31 +08:00
Gantigmaa Selenge 747dc172e8
KIP-1073: Return fenced brokers in DescribeCluster response (#17524)
mplementation of KIP-1073: Return fenced brokers in DescribeCluster response.
Add new unit and integration tests for describeCluster.

Reviewers: Luke Chen <showuon@gmail.com>
2024-12-13 10:58:11 +08:00
Colin Patrick McCabe 65820acad2
MINOR: disable some rebootstrap tests, convert the others to KRaft (#17765)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-13 01:59:20 +08:00
TengYao Chi 772aa241b2
KAFKA-18136: Remove zk migration from code base (#18016)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-12 18:34:29 +01:00
Sushant Mahajan 4c5ea05ec8
KAFKA-18058: Share group state record pruning impl. (#18014)
In this PR, we've added a class ShareCoordinatorOffsetsManager, which tracks the last redundant offset for each share group state topic partition. We have also added a periodic timer job in ShareCoordinatorService which queries for the redundant offset at regular intervals and if a valid value is found, issues the deleteRecords call to the ReplicaManager via the PartitionWriter. In this way the size of the partitions is kept manageable.

Reviewers: Jun Rao <junrao@gmail.com>, David Jacot <djacot@confluent.io>, Andrew Schofield <aschofield@confluent.io>
2024-12-12 07:38:03 +00:00
Mickael Maison fac8333f8d
MINOR: Remove ToolsUtils.scala (#18120)
Reviewers: Christo Lolov <lolovc@amazon.com>
2024-12-11 16:42:05 +00:00
Mickael Maison 7591868aea
KAFKA-18179: Move AsyncOffsetReadFutureHolder to storage module (#18095)
Reviewers: Christo Lolov <lolovc@amazon.com>
2024-12-11 09:56:47 +00:00
David Jacot 57737a357f
KAFKA-18188; Admin LeaveGroup should allow removing member using consumer protocol by member id (#18116)
The LeaveGroup API is used by the admin client to remove static members or remove all members from the group. The latter does not work because the API does not allow removing a member using the CONSUMER protocol by member id. Moreover, the response should only include the member id if the member id was included in the request. This patch fixes both issues.

Reviewers: Dongnuo Lyu <dlyu@confluent.io>, Christo Lolov <lolovc@amazon.com>, Jeff Kim <jeff.kim@confluent.io>
2024-12-10 23:17:32 -08:00
Justine Olshan 3cf8745243
MINOR: Add clientTransactionVersion to AddPartitionsToTxn requests and persist the value across transitions (#18086)
We can better keep track of which transactions use TV_2 by storing this information in the clientTransactionVersion field and persisting it across state transitions. Also updated some logging and equality code to include this information.

Added a test to ensure version persists. There aren't many TV2 transitions that don't specify TV, but I did test the InitProducerId + epoch overflow case.

Reviewers: Artem Livshits <alivshits@confluent.io>, Jeff Kim <jeff.kim@confluent.io>
2024-12-10 12:59:01 -08:00
Lianet Magrans b99c22770a
regex integration tests (#18079)
Reviewers: David Jacot <djacot@confluent.io>
2024-12-10 09:01:02 -05:00
PoAn Yang c8380ae779
KAFKA-17750: Extend kafka-consumer-groups command line tool to support new consumer group (part 2) (#18034)
* Add fields `groupEpoch` and `targetAssignmentEpoch` to `ConsumerGroupDescription.java`.
* Add fields `memberEpoch` and `upgraded` to `MemberDescription.java`.
* Add assertion to `PlaintextAdminIntegrationTest#testDescribeClassicGroups` to make sure member in classic group returns `upgraded` as `Optional.empty`.
* Add new case `testConsumerGroupWithMemberMigration` to `PlaintextAdminIntegrationTest` to make sure migration member has correct `upgraded` value. Add assertion for `groupEpoch`, `targetAssignmentEpoch`, `memberEpoch` as well.

Reviewers: David Jacot <djacot@confluent.io>

Signed-off-by: PoAn Yang <payang@apache.org>
2024-12-10 05:02:20 -08:00
Mickael Maison 08aa8ec3bf
KAFKA-18178 Remove ZkSecurityMigrator (#18092)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-10 16:00:08 +08:00
Kirk True e5598a9d8c
KAFKA-18040; fix for test that ensures produce during follower shutdown (#18108)
Test lacked the proper configuration for the offset topic replication. As a result, when the follower was shut down, the coordinator did not failover properly.

Reviewers: TaiJuWu <tjwu1217@gmail.com>, David Jacot <djacot@confluent.io>
2024-12-10 08:01:51 +01:00
kevin-wu24 3a9777a667
KAFKA-14619; KRaft validate snapshot id are at batch boundaries (#17500)
When KafkaRaftClient receives a request to create a snapshot with end offset that is not aligned to a batch boundary, do not create a misaligned snapshot and instead log at info level and throw an IllegalArgumentException.

Checking that the end offset is at a batch boundary is performed by reading the log at snapshotId.offset() and checking whether the offset in question is the base offset of the returned batch.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2024-12-09 11:38:00 -05:00
Oleksandr K. 54003fe00d
MINOR: Extract some codeblocks as methods to simplify readability (#18017)
Reviewers: Andrew Schofield <aschofield@confluent.io>
2024-12-09 15:25:33 +00:00
Ken Huang d76238a18f
KAFKA-17696 New consumer background operations unaware of metadata errors (#17440)
Reviewers: Kirk True <ktrue@confluent.io>, Lianet Magrans <lmagrans@confluent.io>
2024-12-09 09:31:14 -05:00
Sean Quah 9ae1b0f017
KAFKA-18134; Disallow group upgrades when custom assignors are used (#18046)
Disallow upgrades from classic groups to consumer groups when any member's assignment has non-empty userData.

Reviewers: David Jacot <djacot@confluent.io>
2024-12-09 00:39:22 -08:00
Apoorv Mittal ee4264439d
KAFKA-18129 Simplifying share partition maybeInitialize code (#18093)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-09 03:02:10 +08:00
Andrew Schofield f2e1a4ca8b
KAFKA-17548: Improve group listing integration test (#18081)
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
2024-12-07 18:42:36 +00:00
Mickael Maison 4630628701
KAFKA-18144 Move the storage exceptions out of the core module (#18021)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-07 22:43:00 +08:00
Mickael Maison e255433374
KAFKA-18162 Move LocalLogTest to storage module (#18057)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-07 10:19:56 +08:00
Mickael Maison 417bd22a06
KAFKA-18163: Move VerificationGuardTest to storage module (#18058)
Reviewers: Justine Olshan <jolshan@confluent.io>
2024-12-06 10:18:07 -08:00
Abhinav Dixit 15206d5083
KAFKA-18084 Added usage for rollback state while SharePartition acquires records (#17965)
Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-07 01:32:08 +08:00
Calvin Liu 755adf8a56
KAFKA-14563: RemoveClient-Side AddPartitionsToTxn Requests (#17698)
Removes the client side AddPartitionsToTxn/AddOffsetsToTxn calls so that the partition is implicitly added as part of KIP-890 part 2. 

This change also requires updating the valid state transitions. The client side can not know for certain if a partition has been added server side when the request times out (partial completion). Thus for TV2, the transition to PrepareAbort is now valid for Empty, CompleteCommit, and CompleteAbort. 

For readability, the V1 and V2 endTransaction methods have been separated. 

Reviewers: Artem Livshits <alivshits@confluent.io>, Justine Olshan <jolshan@confluent.io>, Ritika Reddy <rreddy@confluent.io>
2024-12-06 09:00:04 -08:00
Apoorv Mittal ccca9f146e
KAFKA-17945, KAFKA-17944: Handling leader epoch error for offset reset APIs (#17979)
The PR sends known leader epoch while fetch offset information. If API throws exception then PR adds way to handle the exceptions and surface same to make decision for handling share partition itself.

Reviewers: Abhinav Dixit <adixit@confluent.io>, Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>
2024-12-06 09:55:38 +00:00
Colin Patrick McCabe 81ada393d3
MINOR: convert ConsumerBounceTest to KRaft (#17997)
Reviewers: TengYao Chi <kitingiao@gmail.com>, TaiJuWu <tjwu1217@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-06 15:25:48 +08:00
Andrew Schofield e7d986e48c
KAFKA-17550: DescribeGroups v6 exploitation (#17706)
This PR introduces the DescribeGroups v6 API as part of KIP-1043. This adds an error message for the described groups so that it is possible to get some context on the error. It also changes the behaviour for when the group ID cannot be found but returning error code GROUP_ID_NOT_FOUND rather than NONE.

Reviewers: David Jacot <djacot@confluent.io>
2024-12-05 23:12:24 -08:00
Yung bdd36f0adb
KAFKA-18140 Remove ZooKeeperMainWithTlsSupportForKafka (#18022)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-06 04:11:10 +08:00
Abhinav Dixit e663c3857a
KAFKA-18129: `SharePartition#maybeInitialize` should complete the future outsize the write lock (#18053)
Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>
2024-12-05 20:01:00 +00:00
Dongnuo Lyu e30edb3eff
KAFKA-18052: Decouple the dependency of feature stable version to the metadata version (#17886)
Currently the validation of feature upgrade relies on the supported version range generated during registration. For a given feature, its max supported feature version in production is set to be the default version value (the latest feature version with bootstrap metadata value smaller or equal to the latest production metadata value).

This patch introduces a LATEST_PRODUCTION value independent from the metadata version to each feature so that the highest supported feature version can be customized by the feature owner.

The change only applies to dynamic feature upgrade. During formatting, we still use the default value associated the metadata version.

Reviewers: Justine Olshan <jolshan@confluent.io>, Jun Rao <junrao@gmail.com>
2024-12-05 11:07:47 -08:00
Yung fa54065298
KAFKA-18086: Enable propagation of the error message when writing state (#17980)
* KAFKA-18086: Enable propagation of the error message when writing state

* Propagate the error message in the writing state when calling SharePartitionManager.acknowledge and SharePartitionManager.releaseSession, and add corresponding tests to verify that the expected error message is propagated.

* Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>
2024-12-05 17:48:26 +00:00
David Jacot e99561e1f3
KAFKA-17593; [10/N] Remove resolved regular expressions when unsubscribed (#17976)
This patch does a few things:
1) It cleans up resolved regular expressions when they are unsubscribed from. It covers the regular leave/fenced paths for the new protocol and it also covers the LeaveGroup API as new members could be removed via the admin API.
2) It ensures that tombstones for  resolved regular expressions are generated on the conversion patch from consumer to classic group.
3) It fixes [KAFKA-18116](https://issues.apache.org/jira/browse/KAFKA-18116) because I faced the same issue while working on the LeaveGroup API. It adds an integration test for this case too.

Reviewers: Dongnuo Lyu <dlyu@confluent.io>, Jeff Kim <jeff.kim@confluent.io>
2024-12-04 23:41:37 -08:00
Ken Huang 2b43c49f51
KAFKA-18050 Upgrade the checkstyle version to 10.20.2 (#17999)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-05 10:59:18 +08:00
Apoorv Mittal 2d39d5be64
KAFKA-17783: Adding listeners to remove share partition on partition changes (#17796)
Reviewers: Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>
2024-12-04 15:43:37 -08:00
Lianet Magrans bd0ea70912
KAFKA-18096: Allow join with regex if no matching topics (#18024)
Reviewers: David Jacot <djacot@confluent.io>
2024-12-04 11:35:42 -05:00
Andrew Schofield 423a6353a8
KAFKA-18025: Rework acquisition lock timeout test (#17985)
Additional work on ShareConsumerTest.testAcquisitionLockTimeoutOnConsumer. First, mark the test as flaky since it fails occasionally and it would be best to get a decent number of passes before assuming it's no longer flaky. Then, change the assertions so that the test checks which records were received before it counts them (we might get too many records because the wrong records are being returned, or because records are actually being duplicated). The rare failures appear to be related to returning too many records, so it would be good to see whether we can learn more about the duplication.

There are a couple of other improvements too. Reducing the number of share-group state partitions so we don't have the overhead of creating 50 partitions when a few will do. Changing the warm-up logic since that has been observed very occasionally to assert.

Reviewers: ShivsundarR <shr@confluent.io>,  Manikumar Reddy <manikumar.reddy@gmail.com>
2024-12-04 20:58:55 +05:30
Lianet Magrans f60382bf21
KAFKA-18127 Validate SubscriptionPattern used on v0 HB (#17989)
Reviewers: David Jacot <djacot@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-04 19:55:12 +08:00
Peter Lee 095bd0a6d4
KAFKA-18101: Merge duplicate assertFutureThrows and assertFutureExceptionTypeEquals (#17991)
Reviewers: Ziming Deng<dengziming1993@gmail.com>,  Chia-Ping Tsai<chia7712@gmail.com>,  TaiJuWu<tjwu1217@gmail.com>.
2024-12-04 14:15:03 +08:00
Justine Olshan dbae448a05
KAFKA-18137: Unloading transaction state incorrectly removes loading partitions (#18011)
When there is a become follower transition on a transaction coordinator state partition, we intend to unload the state partition. However, we pass the new epoch to the method that does the unloading. In that method, we create a `TransactionPartitionAndLeaderEpoch` object comprising of the topic partition and the epoch that we use as a key to remove the partition from loading. However, we wouldn't ever expect to see this epoch in that map since we only load on the leader. See the code snippet: d00f0ecf1a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala (L602)

We could have a partition load after the unloading occurs, and that partition will be stuck storing stale state on the broker until it restarts. While this may not immediately cause a correctness issue, we should try to properly clean up state.

Check that the epoch is less than the new epoch when removing the partition from loadingPartitions.

Added a test that failed before this change was made.

Reviewers: Artem Livshits <alivshits@confluent.io>, Jeff Kim <jeff.kim@confluent.io>
2024-12-03 14:51:07 -08:00
PoAn Yang fe88232b07
KAFKA-17750 Extend kafka-consumer-groups command line tool to support new consumer group (part 1) (#17958)
1) Bump validVersions of ConsumerGroupDescribeRequest.json and ConsumerGroupDescribeResponse.json to "0-1".

2) Add MemberType field to ConsumerGroupDescribeResponse.json. Default value is -1 (unknown). 0 is for classic member and 1 is for consumer member.

3) When ConsumerGroupMember#useClassicProtocol is true, return MemberType field as 0. Otherwise, return 1.

Reviewers: David Jacot <djacot@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-04 06:08:39 +08:00
Kuan-Po Tseng ac8b3dfbf0
KAFKA-18060 new coordinator does not handle TxnOffsetCommitRequest with empty member id when using CONSUMER group (#17914)
There are two issues in KAFKA-18060:

1) New coordinator can't handle the TxnOffsetCommitRequest with empty member id, and TxnOffsetCommitRequest v0-v2 do definitely has an empty member ID, causing ConsumerGroup#validateOffsetCommit to throw an UnknownMemberIdException. This prevents the old producer from calling sendOffsetsToTransaction. Note that TxnOffsetCommitRequest versions v0-v2 are included in KIP-896, so it seems the new coordinator should support that operations

2) The deprecated API Producer#sendOffsetsToTransaction does not use v0-v2 to send TxnOffsetCommitRequest with an empty member ID. Unfortunately, it has been released for a while. Therefore, the new coordinator needs to handle TxnOffsetCommitRequest with an empty member ID for all versions.

Taken from the two issues above, we need to handle empty member id in all API versions when new coordinator are dealing with TxnOffsetCommitRequest.

Reviewers: David Jacot <djacot@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-04 02:55:19 +08:00
Abhinav Dixit 180112a4a9
KAFKA-18084 Added write locks in SharePartition where locks were async calls were being made (#17957)
Reviewers: Andrew Schofield <aschofield@confluent.io>, poorv Mittal <apoorvmittal10@gmail.com>, Sushant Mahajan <sushant.mahajan88@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-04 01:46:29 +08:00
David Jacot 275b995bf2
KAFKA-18095; Allow a member to join without subscription under new consumer protocol (#18003)
This patch relaxes requiring non-empty subscribed names and regex in the full heartbeat request. Without this, a consumer using client side regexes may not be able to join the group when the regex does not match any topics yet and this is inconsistent with the old protocol. Relaxing the subscribed regex is not strictly required but it seems better to keep it consistent.

Reviewers: Lianet Magrans <lmagrans@confluent.io>
2024-12-03 02:11:36 -08:00
Manikumar Reddy f87c04854b
KAFKA-18013: Add integration test for duration based reset strategy in KafkaConsumer (#18008)
Add new test case to PlaintextConsumerFetchTest

Reviewers: Lianet Magrans <lmagrans@confluent.io>
2024-12-03 10:33:57 +05:30
Chia-Chuan Yu d00f0ecf1a
KAFKA-18124 Remove zk migration from `RaftManagerTest`, `BrokerLifecycleManagerTest`, `KafkaConfigTest`, and `ReplicaManagerTest` (#17990)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-03 03:27:12 +08:00
TengYao Chi 6fd951a9c0
KAFKA-17610 Drop alterConfigs (#18002)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-02 23:26:06 +08:00
Yung 45c094ed23
KAFKA-18130 Remove kafka.utils.VersionInfo (#18001)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-02 15:36:53 +08:00
PoAn Yang 60ce83b2c6
KAFKA-18123 Fix flaky DynamicBrokerReconfigurationTest#testThreadPoolResize (#17986)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-01 20:58:02 +08:00
dengziming 615f1a0bf9
KAFKA-16181: Use incrementalAlterConfigs when updating broker configs by kafka-configs.sh (#15304)
This PR implement KIP-1011, kafka-configs.sh now uses incrementalAlterConfigs API to alter broker configurations instead of the deprecated alterConfigs API, and it will fall directly if the broker doesn't support incrementalAlterConfigs.

Reviewers: David Jacot <djacot@confluent.io>, OmniaGM <o.g.h.ibrahim@gmail.com>.
2024-12-01 18:32:02 +08:00
Lianet Magrans 6237325fb1
KAFKA-15561 [5/N]: Integration tests for new subscribe API with Re2J pattern (#17964)
- integration tests for new subscribe api with RE2J pattern
- fix to ensure all topics are included in metadata requests when consumer is subscribed to RE2J pattern

Reviewers: David Jacot <djacot@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-11-30 01:02:39 +08:00
Ken Huang 9d23f89e05
KAFKA-17338 ConsumerConfig should prevent using partition assignors with CONSUMER group protocol (#16899)
Reviewers: Kirk True <ktrue@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, Lianet Magrans <lmagrans@confluent.io>
2024-11-29 09:36:29 -05:00
Colin Patrick McCabe b8c8e0c713
MINOR: Convert DynamicBrokerReconfigurationTest to KRaft (#17905)
Convert testUncleanLeaderElectionEnable and testMetricsReporterUpdate to KRaft.

Remove testAdvertisedListenerUpdate, testAddRemoveSslListener, testAddRemoveSaslListeners since we no longer support dynamically adding or removing network listeners when in KRaft mode.

Make TestUtils.ensureConsistentKRaftMetadata robust against brokers that don't have sharedServer.loader initialized yet (or have shut down).

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-28 11:59:38 -08:00
HYUNSANG HAN (한현상, Travis) 700bdd5fee
KAFKA-17997 Remove deprecated config log.message.timestamp.difference.max.ms (#17928)
Reviewers: Divij Vaidya <diviv@amazon.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-11-29 03:15:46 +08:00
Peter Lee b08b64c2d8
KAFKA-18098 add kraft support to testReplicaPlacementAllServers and testReplicaPlacementPartialServers (#17955)
Reviewers: Yung <yungyung7654321@gmail.com>, Colin P. McCabe <cmccabe@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2024-11-28 05:21:47 +08:00
Colin Patrick McCabe cdf3aab661
MINOR: remove zk from several tests (#17949)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-28 04:52:04 +08:00
TaiJuWu 486f65e8c6
KAFKA-18100 `Using` block suppresses all errors (#17954)
https://github.com/apache/kafka/pull/15881 changed our tests to utilize `using` blocks. But these blocks don't throw any errors, so if there is a failed assertion within the block, the test will still pass. 

We should either check the failure using a corresponding `match` block with Success(_) and Failure(e), use `using.resource`, or use try/finally blocks to clean up resources.

See https://www.scala-lang.org/api/3.0.2/scala/util/Using$.html

Co-authored-by: frankvicky <kitingiao@gmail.com>

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-28 03:25:19 +08:00
David Jacot aae42ef656
KAFKA-17593; [9/N] Mark ConsumerGroupHeartbeat API v1 as stable (#17961)
Reviewers: Lianet Magrans <lmagrans@confluent.io>
2024-11-27 13:03:46 -05:00
Yung 434fe7c26e
KAFKA-18081 Remove isKRaftTest from the kraft-only tests (#17934)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-27 18:37:43 +08:00
Calvin Liu 2b2b3cd355
KAFKA-18062: use feature version to enable ELR (#17867)
Replace the ELR static config with feature version.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-11-26 14:40:23 -08:00
PoAn Yang 98d47f47ef
KAFKA-18028 the effective kraft version of --no-initial-controllers should be 1 rather than 0 (#17836)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-27 01:45:11 +08:00
Ken Huang b42efc7dc2
KAFKA-18049: Upgrade the caffeine version to 3.1.1 (#17879)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-11-26 18:00:50 +01:00