Commit Graph

316 Commits

Author SHA1 Message Date
Sanskar Jhajharia b2f0d92c45
[MINOR] Fix the docs for share group metric functions (#19023)
The last commit in this class mistakenly described the functions to be
for Streams Groups. Just a minor update.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>, Sushant Mahajan <smahajan@confluent.io>
2025-02-25 10:16:00 +00:00
David Jacot 2124511431
MINOR: Rearrange configs in GroupCoordinatorConfigs (#18970)
I was looking into GroupCoordinatorConfigs to review configurations that
we will ship with Apache Kafka 4.0. I found out that it was pretty
disorganised. This patch cleans up the format and re-groups the
configurations which are related.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-21 13:20:58 +01:00
Sushant Mahajan c2cb543a1e
KAFKA-18629: Delete share group state RPC group coordinator impl. [3/N] (#18848)
* In this PR, we have added GC side impl to call the delete state share
coord RPC using the persister.
* We will be using the existing `GroupCoordinatorService.deleteGroups`.
The logic will be modified as follows:
* After sanitization, we will call a new
`runtime.scheduleWriteOperation` (not read for consistency) with
callback `GroupCoordinatorShard.sharePartitions`. This will return a Map
of share partitions of the groups which are of SHARE type. We need to
pass all groups as WE CANNOT DETERMINE the type of the group in the
service class.
* Then using the map we will create requests which could be passed to
the persister and make the appropriate calls.
* Once this future completes, we will continue with the existing flow of
group deletion.
* If the group under inspection is not share group - the read callback
should return an empty map.
* Tests have been added wherever applicable.

Reviewers: David Jacot <djacot@confluent.io>, Andrew Schofield <aschofield@confluent.io>
2025-02-21 12:13:16 +00:00
Sushant Mahajan c89fd2bff6
KAFKA-18828: Update share group metrics per new init and call mechanism. (#18962)
* Due to recent changes in the way group count metrics are initialized
and updated, the current share group count code has become obsolete as
well as non-functional.
* The update method for the share group count which should be called
from `ShareGroup` cannot be called either. This is because the
constructor has been changed to NOT accept the
`GroupCoordinatorShardMetrics` ref.
* In this PR, we remedy the situation by bringing share group count code
at par with consumer and streams groups.
* Additionally the metric name for share groups with group state
attributes was not aligned with streams and consumer groups as mentioned
in https://github.com/apache/kafka/pull/17011#discussion_r1960309578.
This PR aligns them too.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-02-20 10:23:37 +00: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
David Jacot d847d5c464
MINOR: Tweak default group coordinator config & upgrade notes (#18948)
This patch changes the default value of `group.coordinator.threads` to `4` and sets it priority to `HIGH`. This change makes it consistent with how we handle `num.network.threads` and `num.io.threads`. The patch also tweaks the upgrade notes.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2025-02-18 20:05:52 +01:00
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
Andrew Schofield 6c14f64245
MINOR: Rename NoOpShareStatePersister for consistency (#18933)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-18 14:07:59 +00:00
Lucas Brutschy d0c65a1fd2
KAFKA-18730: Add replaying streams group state from offset topic (#18809)
Adds streams group to the GroupMetadataManager, and implements loading
the records from the offset topic into state. The state also contains
two timers (rebalance timeout and session timeout) that are started
after the group coordinator has been loaded.

Reviewers: Bruno Cadonna <bruno@confluent.io>, Bill Bejeck <bill@confluent.io>
2025-02-17 16:13:21 +01: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
Lucas Brutschy c70b7c4b9e
KAFKA-18323: Add StreamsGroup class (#18729)
Implements a memory model for representing streams groups in the group coordinator, as well as group count and rebalance metrics.

Reviewers: Bill Bejeck <bill@confluent.io>, Bruno Cadonna <bruno@confluent.io>
2025-02-12 11:01:53 +01:00
David Jacot be89ce5f61
MINOR: Accept specifying consumer group assignors by their short names (#18832)
At the moment, we require specifying builtin server side assignors by their full class name. This is not convenient and also exposed their full class name as part of our public API. This patch changes it to accept specifying builtin server side assignor by their short name (uniform or range) while continuing to accept full class name for customer assignors.

Reviewers: Jeff Kim <jeff.kim@confluent.io>
2025-02-10 02:58:44 -08:00
Sanskar Jhajharia 7dbed2f6e8
[KAFKA-16720] AdminClient Support for ListShareGroupOffsets (2/2) (#18671)
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Sushant Mahajan <smahajan@confluent.io>, Andrew Schofield <aschofield@confluent.io>
2025-02-05 14:38:09 +00:00
Lucas Brutschy 4ca24a7dbf
KAFKA-18325: Add TargetAssignmentBuilder (#18676)
A class to build a new target assignment based on the provided parameters. As a result, it yields the records that must be persisted to the log and the new member assignments as a map.

Compared to the feature branch, I extended the unit tests (testing also standby and warm-up task logic) and adopted simplifications due to the TasksTuple class.

Reviewers: Bruno Cadonna <cadonna@apache.org>, Bill Bejeck <bbejeck@apache.org>
2025-02-03 17:35:28 +01:00
Dongnuo Lyu 1a106e4538
KAFKA-18655: Implement the consumer group size counter with scheduled task (#18717)
During testing we discovered that the empty group count is not updated in group conversion, but when the new group is transition to other state, the empty group count is decremented. This could result in negative empty group count.

We can have a new consumer group count implementation that follows the pattern we did for the classic group count. The timeout task periodically refreshes the metrics based on the current groups soft state.

Reviewers: Jeff Kim <jeff.kim@confluent.io>
2025-02-03 10:50:21 -05:00
David Jacot bf05d2c914
KAFKA-18672; CoordinatorRecordSerde must validate value version (#18749)
CoordinatorRecordSerde does not validate the version of the value to check whether the version is supported by the current version of the software. This is problematic if a future and unsupported version of the record is read by an older version of the software because it would misinterpret the bytes. Hence CoordinatorRecordSerde must throw an error if the version is unknown. This is also consistent with the handling in the old coordinator.

Reviewers: Jeff Kim <jeff.kim@confluent.io>
2025-02-03 02:19:27 -08:00
Alieh Saeedi eb01221dc0
KAFKA-17125: Streams Sticky Task Assignor (#18652)
Implements streams sticky assignor on the broker-side.

Reviewers: Bill Bejeck <bbejeck@apache.org>, Lucas Brutschy <lbrutschy@confluent.io>
2025-02-03 10:43:26 +01:00
David Jacot d19b605210
KAFKA-18320; Ensure that assignors are at the right place (#18750)
The full class name of the assignors if part of our public api. Hence, we should ensure that they are not changed by mistake. This patch adds a unit test verifying them.

Reviewers: Sean Quah <squah@confluent.io>, Jeff Kim <jeff.kim@confluent.io>
2025-01-31 07:51:28 -08:00
Lucas Brutschy 56e50120be
KAFKA-18621: Add StreamsCoordinatorRecordHelpers (#18669)
A class with helper methods to create records stored in the __consumer_offsets topic.

Compared to the feature branch, I added unit tests (most functions were not tested) and adopted the new interface for constructing coordinator records introduced by David.

Reviewers: Bruno Cadonna <cadonna@apache.org>
2025-01-30 09:28:45 +01:00
Lucas Brutschy aea699bdef
KAFKA-18324: Add CurrentAssignmentBuilder (#18476)
Implements the current assignment builder, analogous to the current assignment builder of consumer groups. The main difference is the underlying assigned resource, and slightly different logic around process IDs: We make sure to move a task only to a new client, once the task is not owned anymore by any client with the same process ID (sharing the same state directory) - in any role (active, standby or warm-up).

Compared to the feature branch, the main difference is that I refactored the separate treatment of active, standby and warm-up tasks into a compound datatype called TaskTuple (which is used in place of the more specific Assignment class). This also has effects on StreamsGroupMember.

Reviewers: Bruno Cadonna <cadonna@apache.org>, Bill Bejeck <bbejeck@apache.org>
2025-01-23 17:35:03 +01:00
Andrew Schofield 9da516b1a9
KAFKA-18392: Ensure client sets member ID for share group (#18649)
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Lianet Magrans <lmagrans@confluent.io>
2025-01-22 08:57:40 +00:00
David Jacot b368c38684
KAFKA-18302; Update CoordinatorRecord (#18512)
This patch does a few things:
1) Replace ApiMessageAndVersion by ApiMessage in CoordinatorRecord for the key
2) Leverage the fact that ApiMessage exposes the apiKey. Hence we don't need to specify the key anymore.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-01-21 18:11:26 +01:00
Lucas Brutschy 7d39ba1ef8
KAFKA-18311: Internal Topic Manager (5/5) (#18442)
The internal topic manager takes a requested topology and returns a configured topology, as well as any internal topics that need to be created.

Shares with the client-side "InternalTopicManager" the name only.

Reviewers: Bruno Cadonna <cadonna@apache.org>
2025-01-21 09:42:07 +01:00
David Jacot 76bf38a4fd
KAFKA-18604; Update transaction coordinator (#18636)
This patch updates the transaction coordinator record to use the new coordinator record definition.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-01-21 08:36:23 +01:00
Dongnuo Lyu 9026b6c6c0
KAFKA-18318: Add logs for online/offline migration indication (#18406)
Add some logs when offline/online happens.

Reviewers: David Jacot <djacot@confluent.io>
2025-01-17 05:30:32 -08:00
Sean Quah 65b93fa8d5
KAFKA-18150; Downgrade group on classic leave of last consumer member (#18224)
Reviewers: David Jacot <djacot@confluent.io>
2025-01-17 05:27:39 -08:00
Lianet Magrans a9688b9507
MINOR: Improve doc for config group.consumer.max.size (#18533)
Reviewers: David Jacot <david.jacot@gmail.com>
2025-01-14 21:36:14 +01: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
Lucas Brutschy b9a952df6c
KAFKA-18311: Enforcing copartitioned topics (4/N) (#18397)
A simplified port of "CopartitionedTopicsEnforcer" from the client-side to the group coordinator.

This class is responsible for enforcing the number of partitions in copartitioned topics. For each copartition group, it checks whether the number of partitions for all topics in the group is the same, and enforces the right number of partitions for repartition topics whose number of partitions is not enforced by the topology.

Compared to the client-side version, the implementation uses immutable data structures, and returns the computed number of partitions instead of modifying mutable data structures and calling the admin client.

Reviewers: Bruno Cadonna <cadonna@apache.org>
2025-01-10 12:31:09 +01:00
David Jacot 87334e6c2e
KAFKA-18308; Update CoordinatorSerde (#18455)
This patch updates the GroupCoordinatorSerde and the ShareGroupCoordinatorSerde to leverage the CoordinatorRecordType to deserialize records. With this, newly added record are automatically picked up. In other words, the serdes work with all defined records without doing anything.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-01-10 11:17:30 +01:00
Sean Quah fc48b8166b
MINOR: Clean up classic group tests (#18473)
Use `List.of` and `Set.of` where possible.

Reviewers: David Jacot <djacot@confluent.io>
2025-01-09 23:04:45 -08:00
Lucas Brutschy 3bda9f817d
KAFKA-18311: Configuring repartition topics (3/N) (#18395)
A simplified port of "RepartitionTopics" from the client-side to the group coordinator.

Compared to the client-side version, the implementation uses immutable data structures, and returns the computed number of partitions instead of modifying mutable data structures and calling the admin client.

Reviewers: Bruno Cadonna <cadonna@apache.org>
2025-01-09 13:56:37 +01:00
Bruno Cadonna 11459ae7e9
KAFKA-18453: Add StreamsTopology class to group coordinator (#18446)
Adds a class that represent the topology of a Streams group sent by a Streams client in the Streams group heartbeat during initialization to the group coordinator.

This topology representation is used together with the partition metadata on the broker to create a configured topology.

Reviewer: Lucas Brutschy <lbrutschy@confluent.io>
2025-01-09 13:16:03 +01:00
Bruno Cadonna 624dd45809
KAFKA-18321: Add StreamsGroupMember, MemberState and Assignment classes (#18276)
* KAFKA-18321: Add StreamsGroupMember, MemberState and Assignment classes

This commit adds the classes to represent a Streams group member in the
consumer coordinator.

Reviewers: Bill Bejeck <bill@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>
2025-01-08 17:26:41 +01:00
Lucas Brutschy 2521aee7d4
KAFKA-18311: Configuring changelog topics (2/N) (#18379)
A simplified port of "ChangelogTopics" from the client-side to the group coordinator

Compared to the client-side version, the implementation uses immutable data structures, and returns the computed number of partitions instead of modifying mutable data structures and calling the admin client.

Reviewers: Bruno Cadonna <cadonna@apache.org>
2025-01-06 17:17:28 +01:00
Lucas Brutschy 35489bfca3
KAFKA-18311: Add internal datastructure for configuring topologies (1/N) (#18268)
Clients in the Streams Rebalance Protocol send an "unconfigured" representation of the topology to the broker. That is, the number of input topics and (some) internal topics is not fixed, regular expressions are not resolved. The broker takes this description of the topology and, together with the current state of the topics on the broker, derives a ConfiguredTopology. The configured topology is what is being returned from StreamsGroupDescribe, and has all number of partitions defined, and regular expressions resolved. The configured topology also contains missing internal topics that need to be created, and potentially configuration errors, such as missing source topics.

In this change, we add the internal data structures for representing the configured topology. They differ in some details from the data structures used in the RPCs. Most importantly, they can be evolved independently of the public interface.

Reviewers: Bruno Cadonna <cadonna@apache.org>
2025-01-06 13:51:00 +01:00
Ismael Juma d6f24d3665
Use `instanceof` pattern to avoid explicit cast (#18373)
This feature was introduced in Java 16.

Reviewers: David Arthur <mumrah@gmail.com>, Apoorv Mittal <apoorvmittal10@gmail.com>
2025-01-02 09:32:51 -08:00
Lucas Brutschy dd0fd55316
KAFKA-18319: Add task assignor interfaces (#18270)
Introduces interfaces for defining task assignors. Task assignors are pure functions, mapping the state of the group and a topology to a target assignment. We include a mock assignor, which we will be able to use when testing / benchmarking without the complexities of the sticky task assignor and the high-availability task assignor. We may remove the mock assignor in before the streams rebalance protocol goes GA.

The consumer groups introduce these interfaces to establish a clear separation between the group coordinator code and the pluggable assignors, which may live outside the group coordinator code. We have removed pluggable assignors in KIP-1071, but I think it still makes sense to keep these interfaces for having a clean interface for people to code against. This will pay off, if we plan on making the task assignors pluggable later.

Reviewers: Bill Bejeck <bbejeck@gmail.com>, David Jacot <djacot@confluent.io>
2025-01-02 16:31:35 +01:00
Andrew Schofield 0344f8f5ae
KAFKA-18273: KIP-1099 verbose display share group options (#18259)
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2025-01-02 09:12:52 +00: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
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
Jeff Kim cd08129f3d
MINOR: change the default linger time in the new coordinator (#18274) 2024-12-19 16:54:23 -05:00
Bruno Cadonna 4a6c6d1a1e
KAFKA-18284: Add group coordinator records for Streams rebalance protocol (#18228)
KIP-1071 specifies records that the group coordinator needs to store
into the consumer offset topic to persist the state of a Streams
group. This records are specified in json files from which the actual
classes for the records are generated.

This commit adds the records needed by the group coordinator to store
the state of a Streams group.

Reviewer: Lucas Brutschy <lbrutschy@confluent.io>
2024-12-19 09:03:33 +01:00
David Jacot bc5b627ebb
MINOR: Mark KIP-848's public apis as stable (#18235)
KIP-848 will be release as GA in Apache Kafka 4.0. Hence we need to mark all the related public apis as stable.

Reviewers: Jeff Kim <jeff.kim@confluent.io>
2024-12-17 20:02:23 +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
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
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
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
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
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