Commit Graph

5638 Commits

Author SHA1 Message Date
Apoorv Mittal a6c53d0c37
KAFKA-18878: Added share session cache and delayed share fetch metrics (KIP-1103) (#19059)
The PR implements the ShareSessionCache and DelayedShareFetchMetrics as
defined in KIP-1103.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-03-03 16:44:34 +00:00
Lucas Brutschy a04dd21f26
KAFKA-18613: Auto-creation of internal topics in streams group heartbeat (#18981)
Implements auto-topic creation when handling the streams group
heartbeat.

Inside KafkaApis, the handler for streamsGroupHeartbeat uses the result
of the streams group heartbeat inside the group coordinator to attempt
to create all missing internal topics using AutoTopicCreationManager.
CREATE TOPIC ACLs are checked. The unit tests class
AutoTopicCreationManagerTest is brought back (it was recently deleted
during a ZK removal PR), but testing only the kraft-related
functionality.

Reviewers: Bruno Cadonna <bruno@confluent.io>

### Committer Checklist (excluded from commit message)
- [ ] Verify design and implementation 
- [ ] Verify test coverage and CI build status
- [ ] Verify documentation (including upgrade notes)
2025-03-03 08:48:00 +01:00
Xuan-Zhang Gong ceac4f0a1d
KAFKA-18880 Remove kafka.cluster.Broker and BrokerEndPointNotAvailableException (#19047)
Remove kafka.cluster.Broker and BrokerEndPointNotAvailableException as they were used by zk path.

Reviewers: TengYao Chi <kitingiao@gmail.com>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-02 10:54:32 +08:00
TengYao Chi e0c77140b2
KAFKA-17039 KIP-919 supports for unregisterBroker (#19063)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-01 23:55:35 +08:00
Nick Guo 98bb79e732
KAFKA-17981 add Integration test for ConfigCommand to add config `key=[val1,val2]` (#17771)
Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, TaiJuWu <tjwu1217@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-01 13:15:25 +08:00
Apoorv Mittal 8cf969e00a
KAFKA-18734: Implemented share partition metrics (KIP-1103) (#19045)
The PR implements the SharePartitionMetrics as defined in KIP-1103, with
one change. The metric `FetchLockRatio` is defined as `Meter` in KIP but
is implemented as `HIstogram`. There was a discussion about same on
KIP-1103 discussion where we thought that `FetchLockRatio` is
pre-aggregated but while implemeting the rate from `Meter` can go above
100 as `Meter` defines rate per time period. Hence it makes more sense
to implement metric `FetchLockRatio` as `Histogram`.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-02-28 14:22:27 +00:00
Apoorv Mittal 8b605bd362
MINOR: Removing share partition manager flaky annotation (#19053)
There isn't any flaky test for SharePartitionManager in last 7 days, removing flaky annotation.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-02-28 08:49:59 +00:00
Dongnuo Lyu 36f19057e1
KAFKA-18813: ConsumerGroupHeartbeat API and ConsumerGroupDescribe API must check topic describe (#18989)
This patch filters out the topic describe unauthorized topics from the
ConsumerGroupHeartbeat and ConsumerGroupDescribe response.

In ConsumerGroupHeartbeat, 
- if the request has `subscribedTopicNames` set, we directly check the
authz in `KafkaApis` and return a topic auth failure in the response if
any of the topics is denied.
- Otherwise, we check the authz only if a regex refresh is triggered and
we do it based on the acl of the consumer that triggered the refresh. If
any of the topic is denied, we filter it out from the resolved
subscription.

In ConsumerGroupDescribe, we check the authz of the coordinator
response. If any of the topic in the group is denied, we remove the
described info and add a topic auth failure to the described group.
(similar to the group auth failure)

Reviewers: David Jacot <djacot@confluent.io>, Lianet Magrans
<lmagrans@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>,
Chia-Ping Tsai <chia7712@gmail.com>, TaiJuWu <tjwu1217@gmail.com>,
TengYao Chi <kitingiao@gmail.com>
2025-02-26 13:05:36 -05:00
Lucas Brutschy cb7c54ccd3
KAFKA-18614, KAFKA-18613: Add streams group request plumbing (#18979)
This change implements the basic RPC handling StreamsGroupHeartbeat and
StreamsGroupDescribe. This includes:
 - Adding an option to enable streams groups on the broker
- Passing describe and heartbeats to the right shard of the group
coordinator
- The handler inside the GroupMetadatManager for StreamsGroupDescribe is
fairly trivial, and is included directly in this PR.
- The handler for StreamsGroupHeartbeat is complex and not included in
this PR yet. Instead, a UnsupportedOperationException is thrown.
However, the interface is already defined: The result of a
streamsGroupHeartbeat is a response, together with a list of internal
topics to be created.

The heartbeat implementation inside the `GroupMetadataManager`, which
actually implements the assignment / reconciliation logic, will come in
a follow-up PR. Also, automatic creation of internal topics will be
created in a follow-up PR.

Reviewers: Bill Bejeck <bill@confluent.io>
2025-02-26 16:33:26 +01:00
Abhinav Dixit 4b5a16bf6f
KAFKA-18757: Create full-function SimpleAssignor to match KIP-932 description (#18864)
### About
The current `SimpleAssignor` in AK assigned all subscribed topic
partitions to all the share group members. This does not match the
description given in
[KIP-932](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=255070434#KIP932:QueuesforKafka-TheSimpleAssignor).
Here are the rules as mentioned in the KIP by which the assignment
should happen. We have changed the step 3 implementation here due to the
reasons
[described](https://github.com/apache/kafka/pull/18864#issuecomment-2659266502)
-

1. The assignor hashes the member IDs of the members and maps the
partitions assigned to the members based on the hash. This gives
approximately even balance.
2. If any partitions were not assigned any members by (1) and do not
have members already assigned in the current assignment, members are
assigned round-robin until each partition has at least one member
assigned to it.
3. We combine the current and new assignment. (Original rule - If any
partitions were assigned members by (1) and also have members in the
current assignment assigned by (2), the members assigned by (2) are
removed.)

### Tests
The added code has been verified with unit tests and the already present
integration tests.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>, TaiJuWu <tjwu1217@gmail.com>
2025-02-26 11:02:23 +00:00
José Armando García Sancio 4a8a0637e0
KAFKA-18723; Better handle invalid records during replication (#18852)
For the KRaft implementation there is a race between the network thread,
which read bytes in the log segments, and the KRaft driver thread, which
truncates the log and appends records to the log. This race can cause
the network thread to send corrupted records or inconsistent records.
The corrupted records case is handle by catching and logging the
CorruptRecordException. The inconsistent records case is handle by only
appending record batches who's partition leader epoch is less than or
equal to the fetching replica's epoch and the epoch didn't change
between the request and response.

For the ISR implementation there is also a race between the network
thread and the replica fetcher thread, which truncates the log and
appends records to the log. This race can cause the network thread send
corrupted records or inconsistent records. The replica fetcher thread
already handles the corrupted record case. The inconsistent records case
is handle by only appending record batches who's partition leader epoch
is less than or equal to the leader epoch in the FETCH request.

Reviewers: Jun Rao <junrao@apache.org>, Alyssa Huang <ahuang@confluent.io>, Chia-Ping Tsai <chia7712@apache.org>
2025-02-25 20:09:19 -05:00
Apoorv Mittal df5839a9f4
KAFKA-17351: Improved handling of compacted topics in share partition (2/N) (#19010)
The PR handles fetch for `compacted` topics. The fix was required only
when complete batch disappears from the topic log, and same batch is
marked re-available in Share Partition state cache. Subsequent log reads
will not result the disappeared batch in read response hence respective
batch will be left as available in the state cache.

The PR checks for the first fetched/read batch base offset and if it's
greater than the position from where the read occurred (fetch offset)
then if there exists any `available` batches in the state cache then
they will be archived.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Abhinav Dixit <adixit@confluent.io>
2025-02-25 14:11:39 +00:00
xijiu 1edc30bf30
KAFKA-17836 Move RackAwareTest to server module (#19021)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-25 18:15:34 +08:00
TaiJuWu 1c82b89b4c
KAFKA-18712 Move Endpoint to server module (#18803)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Mickael Maison <mickael.maison@gmail.com>, Christo Lolov <lolovc@amazon.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-02-25 14:02:51 +08:00
PoAn Yang 10873e4210
KAFKA-18281: Kafka is improperly validating non-advertised listeners for routable controller addresses (#18387)
When a cluster is configured with a dynamic controller quorum, KRaft replica's endpoint are computed using the advertised.listeners property and not the quorum.controller.voters property. This change in the configuration makes it difficult to keeping all previous node configurations compatible with the new endpoint discovery functionality.

The least intrusive solution is to rely on Kafka's reverse hostname lookup when the hostname is not specified. The effective advertised controller listener now remove '0.0.0.0' hostname if the endpoint came from the listener configuration and not the advertised.listener configuration.

Reviewers: José Armando García Sancio <jsancio@apache.org>, Alyssa Huang <ahuang@confluent.io>
2025-02-24 21:51:28 -05:00
Nick Guo d23a61738a
KAFKA-17937 Cleanup AbstractFetcherThreadTest (#18900)
- Remove AbstractFetcherThreadWithIbp26Test as it tests unsupported IBP
- cleanup AbstractFetcherThreadTest to remove unreachable paths, variables, and code

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-25 07:45:47 +08:00
Apoorv Mittal 48a506b7b8
KAFKA-18522: Slice records for share fetch (#18804)
The PR handles slicing of fetched records based on acquire response for
share fetch. There could be additional bytes fetched from log but
acquired offsets can be a subset, typically with `max fetch records`
configuration. Rather sending additional bytes of fetched data to client
we should slice the file and wire only needed batches.

Note: If the acquired offsets are within a batch then we need to send
the entire batch within the file record. Hence rather checking for
individual batches, PR finds the first and last acquired offset, and
trims the file for all batches between (inclusive) these two offsets.

Reviewers: Christo Lolov <lolovc@amazon.com>, Andrew Schofield <aschofield@confluent.io>, Jun Rao <junrao@gmail.com>
2025-02-24 09:55:24 -08:00
mingdaoy 289e958c39
MINOR: Fix validateResourceNameIsNodeId's exception message (#19017)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-24 09:30:02 +08:00
Ismael Juma 13cb87c2d0
MINOR: Remove request log space added inadvertently (#19011)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-23 11:30:19 +08:00
Apoorv Mittal 6e45ab7d84
KAFKA-17351: Update tests and acquire API to allow discard batches from compacted topics (1/N) (#18978)
The PR does following:
1.  Adds `fetchOffset` to `acquire` API in SharePartition.
2. Adds a ShareFetchPartitionData class efficiently handle the
propagation of fetchOffset information.
3. Updates SharePartitionTests to make common code so such improvements
does not require all tests changes for future PRs.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-02-22 16:14:09 +00:00
Sushant Mahajan 4f28973bd1
KAFKA-18827: Initialize share state, share coordinator impl. [1/N] (#18968)
In this PR, we have added the share coordinator and KafkaApis side impl
of the intialize share group state RPC.
ref:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-932%3A+Queues+for+Kafka#KIP932:QueuesforKafka-InitializeShareGroupStateAPI

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-02-22 16:12:08 +00:00
Apoorv Mittal f543eac4fe
KAFKA-18733: Implemented fetch ratio and partition acquire time metrics (3/N) (#18959)
PR implements the final set of ShareGroupMetrics,
RequestTopicPartitionsFetchRatio and TopicPartitionsAcquireTimeMs, as
defined in KIP-1103:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1103%3A+Additional+metrics+for+cooperative+consumption

Note: Metric `RequestTopicPartitionsFetchRatio` is calculated as
percentage as Histogram API doesn't record double.


Reviewers: Andrew Schofield <aschofield@confluent.io>, Abhinav Dixit <adixit@confluent.io>
2025-02-21 17:01:39 +00:00
Calvin Liu 8f13e7c207
MINOR: Move the ELR default version to 4.1 (#18954)
- ELR is enabled (ELRV_1) by default if the cluster is created with its bootstrap metadata version >= IBP_4_1_IV0.
- ELRV_1 can be manually enabled iff the metadata version is >= IBP_4_0_IV1.

Reviewers: Ismael Juma <ismael@juma.me.uk>, Colin P. McCabe <cmccabe@apache.org>, David Jacot <djacot@confluent.io>
2025-02-21 16:13:11 +01:00
Shivsundar R 7da1a6cbff
KAFKA-18033: Remove flaky tag in ShareConsumerTest (#18995)
3 tests which were marked flaky in ShareConsumerTest do not have any
failure on trunk since the test was converted to use `ClusterTestExtensions`.

Reviewers: Sushant Mahajan <smahajan@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2025-02-21 13:50:08 +00:00
TengYao Chi 767a62ade6
KAFKA-18737 KafkaDockerWrapper setup functions fails due to storage format command (#18844)
The current Docker Hub documentation for Kafka is based on the use of static voters. Since Kafka 4.0 utilizes dynamic voters, users following the doc of docker hub may encounter unexpected behavior. Due to the limited time available for the 4.0.0 release, a simple and quick solution is to revert to using static voters within the Docker image. This can be achieved by adding a configuration file with static voter definitions to the kafka/docker folder, keeping it separate from the main kafka/config directory. This approach allows us to encourage the use of dynamic voters in typical deployments while maintaining compatibility within the Docker image.

Reviewers: Vedarth Sharma <142404391+VedarthConfluent@users.noreply.github.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-02-21 20:43:41 +08:00
TengYao Chi d31cbf59de
KAFKA-18831 Migrating to log4j2 introduce behavior changes of adjusting level dynamically (#18969)
fix the following behavior changes.

1) in log4j 1, users can't change the logger by parent if the logger is declared by properties explicitly. For example, `org.apache.kafka.controller` has level explicitly in the properties. Hence, we can't use "org.apache.kafka=INFO" to change the level of `org.apache.kafka.controller` to INFO. By contrast, log4j2 allows us to change all child loggers by the parent logger.

2) in log4j2, we can change the level of root to impact all loggers' level. By contrast, log4j 1 can't. 

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-21 16:12:58 +08:00
Calvin Liu 1eecd02ce8
MINOR: Deflake EligibleLeaderReplicasIntegrationTest (#18923)
Make sure to give enough time for the partition ISR updates.

Reviewers: David Jacot <djacot@confluent.io>
2025-02-20 05:14:15 -08:00
Matthias J. Sax 538a60e1b3
MINOR: disallow rawtypes and fail build (#18877)
Cleanup code to avoid rawtype, and add suppressions where necessary.
Change the build to fail on rawtype warning.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2025-02-19 13:11:49 -08:00
Ismael Juma 3a59a526d9
MIINOR: Remove redundant quorum parameter from *AdminIntegrationTest classes (#18965)
Reviewers: Lianet Magrans <lmagrans@confluent.io>
2025-02-19 15:57:47 -05:00
Shivsundar R 3603c8fe35
KAFKA-18829: Added check before converting to IMPLICIT mode (#18964)
Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-02-19 17:34:28 +00:00
Ismael Juma 3dba3125e9
KAFKA-18601: Assume a baseline of 3.3 for server protocol versions (#18845)
3.3.0 was the first KRaft release that was deemed production-ready and also
when KIP-778 (KRaft to KRaft upgrades) landed. Given that, it's reasonable
for 4.x to only support upgrades from 3.3.0 or newer (the metadata version also
needs to be set to "3.3" or newer before upgrading).

Noteworthy changes:
1. `AlterPartition` no longer includes topic names, which makes it possible to
simplify `AlterParitionManager` logic.
2. Metadata versions older than `IBP_3_3_IV3` have been removed and
`IBP_3_3_IV3` is now the minimum version.
3. `MINIMUM_BOOTSTRAP_VERSION` has been removed.
4. Removed `isLeaderRecoverySupported`, `isNoOpsRecordSupported`,
`isKRaftSupported`, `isBrokerRegistrationChangeRecordSupported` and
`isInControlledShutdownStateSupported` - these are always `true` now.
Also removed related conditional code.
5. Removed default metadata version or metadata version fallbacks in
multiple places - we now fail-fast instead of potentially using an incorrect
metadata version.
6. Update `MetadataBatchLoader.resetToImage` to set `hasSeenRecord`
based on whether image is empty - this was a previously existing issue that
became more apparent after the changes in this PR.
7. Remove `ibp` parameter from `BootstrapDirectory`
8. A number of tests were not useful anymore and have been removed.

I will update the upgrade notes via a separate PR as there are a few things that
need changing and it would be easier to do so that way.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>, David Arthur <mumrah@gmail.com>, Colin P. McCabe <cmccabe@apache.org>, Justine Olshan <jolshan@confluen.io>, Ken Huang <s7133700@gmail.com>
2025-02-19 05:35:42 -08:00
xijiu 4c4458c17a
KAFKA-18799 Remove AdminUtils (#18946)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-19 06:25:43 +08:00
PoAn Yang 1132f08c57
KAFKA-18773 Migrate the log4j1 config to log4j 2 for native image and README (#18872)
- update reflection-config.json and resource-config.json to include log4j2 and jackson
- remove unused jackson scala library
- fix the incorrect path of log4j2.yaml
- adopt workaround (--standalone) to make this PR work and it will be fixed by KAFKA-18737)

Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-02-19 00:48:46 +08:00
TaiJuWu 934b0159bb
KAFKA-18089: Upgrade Caffeine lib to 3.1.8 (#18004)
- Fixed the RemoteIndexCacheTest that fails with caffeine > 3.1.1

Reviewers: Luke Chen <showuon@gmail.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
2025-02-18 21:51:38 +05:30
Parker Chang ed366e6b89
MINOR: Align assertFutureThrows method signature with JUnit conventions (#18825)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2025-02-18 15:56:42 +00:00
Mickael Maison 0a2fab9310
KAFKA-14484: Decouple UnifiedLog and RemoteLogManager (#18460)
Reviewers: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Ismael Juma <ismael@juma.me.uk>
2025-02-18 15:10:31 +01:00
Andrew Schofield 6c14f64245
MINOR: Rename NoOpShareStatePersister for consistency (#18933)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-18 14:07:59 +00:00
Chirag Wadhwa 63229a768c
KAFKA-16718 [1/n]: Added DeleteShareGroupOffsets request and response schema (#18927)
Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-02-18 14:06:24 +00:00
Andrew Schofield 385b7ad355
MINOR: Align share group admin authz with consumer group (#18936)
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2025-02-18 09:12:07 +00:00
Kamal Chandraprakash da3643c6b4
KAFKA-18787: RemoteIndexCache fails to delete invalid files on init (#18888)
The stale/invalid files that ends-with ".deleted" and ".tmp" should be cleaned when the broker gets restarted.

- fix the remote-index-cache test to use the logDir instead of topicDir
- fix the flaky test

Reviewers: Luke Chen <showuon@gmail.com>
2025-02-18 12:56:03 +05:30
Apoorv Mittal 06ce3e890b
KAFKA-18733: Updating share group record acks metric (2/N) (#18924)
Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-02-17 18:12:58 +00:00
PoAn Yang 2b6e868538
KAFKA-18784 Fix ConsumerWithLegacyMessageFormatIntegrationTest (#18889)
In PR #18267, we removed old message format for cases in ConsumerWithLegacyMessageFormatIntegrationTest. Although test cases can pass, they don't fulfill original purpose. We can't send old message format since 4.0, so I change cases to append old records by ReplicaManager directly.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-17 20:43:29 +08:00
Andrew Schofield 9b7ad6ec32
MINOR: Mark testQuotaOverrideDelete as flaky (#18925)
Reviewers: poorv Mittal <apoorvmittal10@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-02-17 15:20:35 +08:00
TengYao Chi 5cbe00e375
MINOR: Remove unused member in DynamicBrokerConfig (#18915)
Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-02-17 04:46:25 +08:00
Ming-Yen Chung e828767062
KAFKA-18790 Fix testCustomQuotaCallback (#18906)
Frequently updating the trust store can cause unexpected termination of the AsyncConsumer background thread.

1. To resolve this issue, reuse the same AdminClient instead of recreating it.
2. Add error logging when fail to initialize resources for the consumer network thread.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-15 03:07:59 +08:00
Jimmy Wang 6a6b80215d
KAFKA-16717 [1/2]: Add AdminClient.alterShareGroupOffsets (#18819)
KAFKA-16720 aims to add the support for the AlterShareGroupOffsets AdminClient. Key Changes in the PR:

1. Added handing of alterShareGroupOffsets() in KafkaAdminClient and introduce AlterShareGroupOffsetRequest/AlterShareGroupOffsetResponse/AlterShareGroupOffsetsOptions classes.
2. Corresponding test in KafkaAdminClientTest.
3. Added ALTER_SHARE_GROUP_OFFSETS API (will finish it in next PR and the share coordinator pieces)

Reviewers: poorv Mittal <apoorvmittal10@gmail.com>, Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2025-02-15 02:35:46 +08:00
Apoorv Mittal 53543bcf63
KAFKA-18733: Updating share group metrics (1/N) (#18826)
Reviewers: Sushant Mahajan <smahajan@confluent.io>, Andrew Schofield <aschofield@confluent.io>
2025-02-14 08:48:41 +00:00
陳昱霖(Yu-Lin Chen) 2bbd25841e
KAFKA-18298 Fix flaky testConsumerGroupsDeprecatedConsumerGroupState and testConsumerGroups in PlaintextAdminIntegrationTest (#18513)
It's related to KAFKA-18298 and KAFKA-18297. The root cause of the flaky tests is member rejoin after member removal. To prevent members from rejoining after being removed, before removing group members, calling `consumers.close` in ConsumerThread . This fix also extract the flaky member removal test  to new test `testConsumerGroupWithMemberRemoval`.

Flow of member removal test: 
1. Set 2 static consumer + 1 dynamic consumer
2. Close all consumers.
3. remove one static member
4. remove remaining members
 
Before KIP-1092, the member count is different between ClassicConsumer/AsyncConsumer. (AsyncConsumer will remove dynamic member after consumer closed.)

To get more details, please refer to the discussion under KAFKA-18297 and this PR:
- discussion : [Link](https://issues.apache.org/jira/browse/KAFKA-18297?focusedCommentId=17912537&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17912537)
- review: https://github.com/apache/kafka/pull/18513#pullrequestreview-2589110367

This PR fixed below flaky errors:

1. **PlaintextAdminIntegrationTest#testConsumerGroups**
  a.  `org.opentest4j.AssertionFailedError: expected: <2> but was: <3>` ([Report](https://ge.apache.org/s/lt3lpviv45cns/tests/task/:core:test/details/kafka.api.PlaintextAdminIntegrationTest/testConsumerGroups(String%2C%20String)%5B1%5D?top-execution=1))
  b.  `org.opentest4j.AssertionFailedError: expected: <true> but was: <false>` ([Report](https://ge.apache.org/s/jlxo446xalpoa/tests/task/:core:test/details/kafka.api.PlaintextAdminIntegrationTest/testConsumerGroups(String%2C%20String)%5B1%5D?top-execution=1))

2. **PlaintextAdminIntegrationTest#testConsumerGroupsDeprecatedConsumerGroupState**
  a.  `org.opentest4j.AssertionFailedError: expected: <2> but was: <3>` ([Report](https://ge.apache.org/s/ndoj6s2stb446/tests/task/:core:test/details/kafka.api.PlaintextAdminIntegrationTest/testConsumerGroupsDeprecatedConsumerGroupState(String%2C%20String)%5B1%5D?top-execution=1))
  b. `org.opentest4j.AssertionFailedError: expected: <true> but was: <false>` ([Report](https://ge.apache.org/s/kh3jze2tc5qeu/tests/task/:core:test/details/kafka.api.PlaintextAdminIntegrationTest/testConsumerGroupsDeprecatedConsumerGroupState(String%2C%20String)%5B1%5D?top-execution=1))

Reviewers: David Jacot <djacot@confluent.io>, TaiJuWu <tjwu1217@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-02-14 07:28:45 +08:00
Andrew Schofield 952113e8e0
KAFKA-16720: Support multiple groups in DescribeShareGroupOffsets RPC (#18834)
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
2025-02-13 18:27:05 +00:00
Calvin Liu 9cb271f1e1
KAFKA-18654[2/2]: Transction V2 retry add partitions on the server side when handling produce request. (#18810)
During the transaction commit phase, it is normal to hit CONCURRENT_TRANSACTION error before the transaction markers are fully propagated. Instead of letting the client to retry the produce request, it is better to retry on the server side.

Reviewers: Artem Livshits <alivshits@confluent.io>, Justine Olshan <jolshan@confluent.io>
2025-02-13 09:30:58 -08:00