Commit Graph

698 Commits

Author SHA1 Message Date
David Jacot 64f3ee4c33
KAFKA-17593; [2/N] Update request validation & validate regex (#17651)
This patch does two things:
1) Change the validation of the ConsumerGroupHeartbeat request to accept subscribed topic names and/or subscribed topic regex. At least of them must be set in the first request with epoch 0.
2) Validate the provided regular expression by compiling it.

Co-authored-by: Lianet Magrans <lmagrans@confluent.io>

Reviewers: Jeff Kim <jeff.kim@confluent.io>, Lianet Magrans <lmagrans@confluent.io>
2024-11-04 06:38:09 -08:00
Yung 0799715f2c
KAFKA-17795 Fix the file path of suppressions.xml to run build on windows (#17674)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-03 21:36:11 +08:00
Ken Huang 30b1bdfc74
KAFKA-17835 Move ProducerIdManager and RPCProducerIdManager to transaction-coordinator module (#17562)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-31 02:40:47 +08:00
Nick Telford 571f50817c
KAFKA-17411: Create local state Standbys on start (#16922)
Instead of waiting until Tasks are assigned to us, we pre-emptively
create a StandbyTask for each non-empty Task directory found on-disk.

We do this before starting any StreamThreads, and on our first
assignment (after joining the consumer group), we recycle any of these
StandbyTasks that were assigned to us, either as an Active or a
Standby.

We can't just use these "initial Standbys" as-is, because they were
constructed outside the context of a StreamThread, so we first have to
update them with the context (log context, ChangelogReader, and source
topics) of the thread that it has been assigned to.

The motivation for this is to (in a later commit) read StateStore
offsets for unowned Tasks from the StateStore itself, rather than the
.checkpoint file, which we plan to deprecate and remove.

There are a few additional benefits:

Initializing these Tasks on start-up, instead of on-assignment, will
reduce the time between a member joining the consumer group and beginning
processing. This is especially important when active tasks are being moved
over, for example, as part of a rolling restart.

If a Task has corrupt data on-disk, it will be discovered on startup and
wiped under EOS. This is preferable to wiping the state after being
assigned the Task, because another instance may have non-corrupt data and
would not need to restore (as much).

There is a potential performance impact: we open all on-disk Task
StateStores, and keep them all open until we have our first assignment.
This could require large amounts of memory, in particular when there are
a large number of local state stores on-disk.

However, since old local state for Tasks we don't own is automatically
cleaned up after a period of time, in practice, we will almost always
only be dealing with the state that was last assigned to the local
instance.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Bruno Cadonna <cadonna@apache.org>, Matthias Sax <mjsax@apache.org>
2024-10-29 12:59:25 -07:00
Sushant Mahajan 5f92f60bff
KAFKA-17329: DefaultStatePersister implementation (#17270)
Adds the DefaultStatePersister and other supporting classes for managing share state.

* Added DefaultStatePersister implementation. This is the entry point for callers who wish to invoke the share state RPC API.
* Added PersisterStateManager which is used by DefaultStatePersister to manage and send the RPCs over the network.
* Added code to BrokerServer and BrokerMetadataPublisher to instantiate the appropriate persister based on the config value for group.share.persister.class.name. If this is not specified, the DefaultStatePersister will be used. To force use of NoOpStatePersister, set the config to empty. This is an internal config, not to be exposed to the end user. This will be used to factory plug the appropriate persister.
* Using this persister, the internal __share_group_state topic will come to life and will be used for persistence of share group info.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Jun Rao <junrao@gmail.com>, David Arthur <mumrah@gmail.com>
2024-10-28 14:11:04 -04:00
Mickael Maison 6e88c10ed5
KAFKA-14483 Move LocalLog to storage module (#17587)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-28 20:41:46 +08:00
Dmitry Werner 1eb7644349
KAFKA-16845 Migrate ReplicationQuotasTestRig to new test infra (#17089)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-25 03:33:01 +08:00
Stig Døssing 98b7e4deaf
KAFKA-17574 Allow overriding TestKitNodes baseDirectory (#17225)
This allows shutting down a KafkaClusterTestKit from a JVM shutdown hook without risking error logs because the base directory has already been deleted by the shutdown hook TestUtils.tempDirectory sets up.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-25 01:48:25 +08:00
Kuan-Po Tseng 140d35c545
KAFKA-8779 Fix flaky tests introduced by dynamic log levels (#17382)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-24 21:39:35 +08:00
TengYao Chi 553e6b4c6d
KAFKA-17860 Remove log4j-appender module (#17588)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-24 18:13:30 +08:00
Ken Huang 2ff13976ab
KAFKA-17568 Rewrite TestPurgatoryPerformance by Java (#17246)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-24 02:44:37 +08:00
PoAn Yang 8e54a04723
KAFKA-17536; Ensure clear error message when new consumer used with incompatible cluster (#17411)
In Kafka 4.0, by default the consumer uses the updated consumer group protocol defined in KIP-848. When the consumer is used with a cluster that does not support (or is not configured to use) the new protocol, the consumer will get an unfriendly error about unavailable APIs. Since this error could be the user's first impression when attempting to upgrade to 4.0, we need to make sure that the error is very clear about the remediation steps (set the group.protocol to CLASSIC on the client or upgrade and enable the new protocol on the cluster).

Reviewers: Kirk True <ktrue@confluent.io>, David Jacot <djacot@confluent.io>
2024-10-16 01:30:57 -07:00
Sushant Mahajan 5545d717c3
KAFKA-17633: Add share group record formatter and parser. (#17467)
As part of KIP-932, a new internal topic __share_group_state was introduced. There are 2 types of records which are currently being added in this topic - ShareSnapshotKey/Value and ShareUpdateKey/Value
In light of this, we must make the existing tooling like kafka-console-consumer and kafka-dump-log aware of these records for debugging and introspection purposes.
This PR introduces ShareGroupStateMessageFormatter to be used used with kafka-console-consumer and adds an internal class ShareGroupStateMessageParser in DumpLogSegments.scala.
Unit tests have been added to DumpLogSegmentsTest.scala


Reviewers:  Andrew Schofield <aschofield@confluent.io>,  Manikumar Reddy <manikumar.reddy@gmail.com>
2024-10-15 11:44:15 +05:30
Linsiyuan9 76a1af984b
KAFKA-17746 Replace JavaConverters with CollectionConverters (#17451)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-14 17:13:20 +08:00
Chung, Ming-Yen d74f1f847a
KAFKA-17199 Add UT for TransactionLogConfig and TransactionStateManagerConfig (#17442)
Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-10-12 09:42:54 +08:00
ShivsundarR 374663c64b
KAFKA-17287: Added integration tests for ShareConsumer (#17009)
Reviewers: Andrew Schofield <aschofield@confluent.io>, David Arthur <mumrah@gmail.com>
2024-10-08 10:37:16 -04:00
Federico Valeri 8dbbf5ea76
KAFKA-14705: Move topic filters to tools module (#17387)
After MirrorMaker 1 removal, there are no other modules dependencies for these classes, so we can safely move them to tools module.

Signed-off-by: Federico Valeri <fedevaleri@gmail.com>
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-10-07 16:54:36 +02:00
Dmitry Werner 672c617233
KAFKA-14577: Move ConsoleProducer to tools module (#17157)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Federico Valeri <fedevaleri@gmail.com>
2024-10-07 11:19:59 +02:00
Chia-Ping Tsai f1d6549679
KAFKA-17706 Allow all imports for test-common and test-common-api (#17378)
Reviewers: David Arthur <mumrah@gmail.com>
2024-10-06 14:02:10 +08:00
Bill Bejeck c11a38f9df
KAFKA-17248: KIP-1076 add admin client test helper [1/N] (#17375)
No functional changes, this PR contains a test-helper class for working with AdminClient

Reviewers Matthias Sax <mjsax@apache.org>
2024-10-04 13:58:16 -04:00
Colin Patrick McCabe dbd50ff847
KAFKA-16469: Metadata schema checker (#15995)
Create a schema checker that can validate that later versions of a KRPC schema are compatible with earlier ones.

Reviewers: David Arthur <mumrah@gmail.com>
2024-10-03 12:13:38 -07:00
Chia-Ping Tsai 979740b49d
KAFKA-17589 Move JUnit extensions to test-common module (#17318)
This patch completely removes the compile-time dependency on core for both test and main sources by introducing two new modules.

1) `test-common` include all the common test implementation code (including dependency on :core for BrokerServer, ControllerServer, etc)
2) `test-common:api` new sub-module that just includes interfaces including our junit extension

Reviewers: David Arthur <mumrah@gmail.com>
2024-10-03 10:28:37 +08:00
Chung, Ming-Yen 540fb91103
KAFKA-17258 Migrate AdminFenceProducersIntegrationTest to ClusterTestExtensions framework (#17311)
Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-10-03 00:47:29 +08:00
Sushant Mahajan 7b7eb6243f
KAFKA-17367: Share coordinator persistent batch merging algorithm. [3/N] (#17149)
This patch introduces a merging algorithm for persistent state batches in the share coordinator. 

The algorithm removes any expired batches (lastOffset before startOffset) and then places the rest in a sorted map. It then identifies batch pairs which overlap and combine them while preserving the relative priorities of any intersecting sub-ranges. The resultant batches are placed back into the map. The algorithm ends when no more overlapping pairs can be found.

Reviewers: Andrew Schofield <aschofield@confluent.io>, David Arthur <mumrah@gmail.com>, Apoorv Mittal <apoorvmittal10@gmail.com>, Jun Rao <junrao@gmail.com>
2024-10-02 11:30:51 -04:00
Omnia Ibrahim 1854d4b8a1
KAFKA-14572: Migrate EmbeddedKafkaCluster used by Streams integration tests from EmbeddedZookeeper to KRaft (#17016)
Migrate the EmbeddedKafkaCluster from the EmbeddedZookeeper to KRaft

Reviewers Bill Bejeck <bbejeck@apache.org>
2024-09-27 15:49:12 -04:00
Dimitar Dimitrov bc47ce1a53
MINOR: Fix a race and add JMH bench for HdrHistogram (#17221) 2024-09-27 23:49:10 +09:00
Apoorv Mittal 05366d2fa7
KAFKA-17626: Move common fetch related classes from storage to server-common (#17289)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Arthur <mumrah@gmail.com>
2024-09-26 20:31:28 -04:00
Matthias J. Sax ce5491b7bf
KAFKA-16331: remove KafkaClientSupplier from StreamsProducer (#17259)
With EOSv1 removed, we don't need to create a producer per task, and thus can simplify the code by removing KafkaClientSupplier from the deeply nested StreamsProducer, to simplify the code.

Reviewers: Bill Bejeck <bill@confluent.io>
2024-09-26 17:07:00 -07:00
xijiu 18340c9733
KAFKA-17563 Move `RequestConvertToJson` to server module (#17223)
Reviewers: Chia-Ping Tsai <chia7712@apache.org>
2024-09-27 02:19:47 +08:00
Apoorv Mittal 6abbd548b8
MINOR Refactoring share fetch code (KIP-932) (#17269)
Reviewers: Andrew Schofield <aschofield@confluent.io>, David Arthur <mumrah@gmail.com>
2024-09-26 08:09:31 -04:00
Sushant Mahajan 67f966f348
KAFKA-17469: Moved share external interfaces to share module. (#17262)
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, David Arthur <mumrah@gmail.com>
2024-09-24 12:08:01 -04:00
Mickael Maison f1c011a8b5
KAFKA-14482 Move LogLoader to storage module (#17042)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-17 00:37:49 +08:00
PoAn Yang 98252cae14
KAFKA-16681 Rewrite MiniKDC by Java (#16213)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-14 04:13:40 +08:00
Apoorv Mittal 300e825be7
MINOR Refactored share module classes (#17178)
Reviewers: Andrew Schofield <aschofield@confluent.io>, David Arthur <mumrah@gmail.com>
2024-09-12 13:32:15 -04:00
Abhinav Dixit f466e86bb5
KAFKA-17400: Added share fetch purgatory for delaying share fetch requests (#16969)
Introduced a share fetch purgatory on the broker which delays share fetch requests that cannot be completed instantaneously. Introduced 2 new classes -

DelayedShareFetch - Contains logic to instantaneously complete or force complete a share fetch request on timeout.
DelayedShareFetchKey - Contains the key which can be used to watch the entries within the share fetch purgatory.
ShareFetchUtils - This utility class contains functionalities required for post-processing once the replica manager fetch is completed.
There are many scenarios which can cause a share fetch request to be delayed and multiple scenarios when a delayed share fetch can be attempted to be completed. In this PR, we are only targeting the case when record lock partition limit is reached, the ShareFetch should wait for up to MaxWaitMs for records to be released.

Reviewers: David Arthur <mumrah@gmail.com>, Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>, Jun Rao <junrao@gmail.com>
2024-09-11 11:47:33 -07:00
Sushant Mahajan 821c10157d
KAFKA-17367: Introduce share coordinator [2/N] (#17011)
Introduces the share coordinator. This coordinator is built on the new coordinator runtime framework. It 
is responsible for persistence of share-group state in a new internal topic named "__share_group_state".
The responsibility for being a share coordinator is distributed across the brokers in a cluster. 

Reviewers: David Arthur <mumrah@gmail.com>, Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>
2024-09-09 20:01:24 -04:00
Mickael Maison 839431e591
KAFKA-17468 Move kafka/log/remote/quota classes to storage module (#17074)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-04 02:18:47 +08:00
Kamal Chandraprakash 88b9ff30ad
KAFKA-15859 Introduce remote.list.offsets.request.timeout.ms dynamic config (#17045)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-03 23:06:01 +08:00
Omnia Ibrahim f59d829381
KAFKA-15853 Move TransactionLogConfig and TransactionStateManagerConfig getters out of KafkaConfig (#16665)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-03 18:24:12 +08:00
Mickael Maison c30615e6d7
KAFKA-17430: Move RequestChannel.Metrics/RequestMetrics to server module (#17015)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-03 10:11:47 +02:00
Mickael Maison 1841c07d4a
KAFKA-17449 Move Quota classes to server-common module (#17060)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-31 12:41:34 +08:00
Luke Chen ad4405c8dd
KAFKA-17062: handle dangling "copy_segment_start" state when deleting remote logs (#16959)
The COPY_SEGMENT_STARTED state segments are counted when calculating remote retention size. This causes unexpected retention size breached segment deletion. This PR fixes it by
  1. only counting COPY_SEGMENT_FINISHED and DELETE_SEGMENT_STARTED state segments when calculating remote log size.
  2. During copy Segment, if we encounter errors, we will delete the segment immediately.
  3. Tests added.

Co-authored-by: Guillaume Mallet <>

Reviewers: Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Satish Duggana <satishd@apache.org>, Guillaume Mallet <>
2024-08-29 14:09:55 +08:00
Mickael Maison b9fe9f532f
KAFKA-16972: Move BrokerTopicStats to storage module (#17003)
Reviewers: Luke Chen <showuon@gmail.com>
2024-08-27 11:39:37 +02:00
DL1231 006af8b939
KAFKA-17327; Add support of group in kafka-configs.sh (#16887)
The patch adds support of alter/describe configs for group in kafka-configs.sh.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <djacot@confluent.io>
2024-08-27 02:16:46 -07:00
Sushant Mahajan 1621f88f06
KAFKA-17367: Share coordinator infra classes [1/N] (#16921)
Introduce ShareCoordinator interface and related classes.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>, David Arthur <mumrah@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-08-26 10:09:47 -04:00
xijiu cb19a557e9
KAFKA-17373 Add print.epoch to kafka-console-share-consumer.sh/kafka-console-consumer.sh (#16987)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-26 17:30:57 +08:00
xijiu 76fb18f967
KAFKA-17416 Add a checkstyle rule to suppress all generated code (#16998)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-26 17:24:27 +08:00
TengYao Chi d67c18b4ae
KAFKA-17331 Set correct version for EarliestLocalSpec and LatestTieredSpec (#16876)
Add the version check to client side when building ListOffsetRequest for the specific timestamp:
1) the version must be >=8 if timestamp=-4L (EARLIEST_LOCAL_TIMESTAMP)
2) the version must be >=9 if timestamp=-5L (LATEST_TIERED_TIMESTAMP)

Reviewers: PoAn Yang <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2024-08-25 17:39:28 +08:00
David Arthur a95dfe215b
MINOR: Move generated sources to build directory (#16993)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-25 14:12:51 +08:00
Jeff Kim ced34e3176
KAFKA-16379; Coordinator event queue, processing, flush, purgatory time histograms (#16949)
This patch introduces a wrapper around [HdrHistogram](https://github.com/HdrHistogram/HdrHistogram) to use for group coordinator histograms, event queue time, event processing time, flush time, and purgatory time.

Reviewers: David Jacot <djacot@confluent.io>
2024-08-23 04:53:22 -07:00
Xuan-Zhang Gong a537e716eb
KAFKA-17137 Ensure Admin APIs are properly tested (#16658)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-21 03:34:10 +08:00
Apoorv Mittal 3db6e68c4c
KAFKA-17346: Create :share Gradle module (#16888)
Establishes the new `:share` Gradle module. This module is intended to be used for server-side KIP-932 classes that are not part of the new share group coordinator.

This patch relocates and renames some existing classes. A small amount of compatibility changes were also made, but do not affect any logic.

Reviewers: Andrew Schofield <aschofield@confluent.io>, David Arthur <mumrah@gmail.com>
2024-08-19 16:13:08 -04:00
Steven Xu e24354a21d
KAFKA-15203: Use Classgraph since org.reflections is no longer under maintenance (#16604)
Reviewers: Liam Miller-Cushon <cushon@google.com>, Greg Harris <greg.harris@aiven.io>
2024-08-19 10:46:24 -07:00
Sushant Mahajan 4928b549fa
KAFKA-17330: New share-coordinator module. Added record schemas. (#16877)
What
Introduced a new module share-coordinator to house relevant implementation code and resources.
Modified settings.gradle and build.gradle to accommodate the module.
Added ShareSnapshot[Key, Value], ShareUpdate[Key, Value] message record schemas.
Introduced a trivial impl of ShareCoordinatorShard class to establish dependencies with other modules (:coordinator-common, :metadata). The actual impl for this class will be done in future PRs.

Why
The share coordinator component has been introduced as part of KIP-932 (QFK). This component is will be responsible for managing persistence for various data related to share partitions into a dedicated internal topic.
To keep all this functionality contained, we want to create a separate module in line with group and transaction coordinators.


Reviewers:  Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>,  Manikumar Reddy <manikumar.reddy@gmail.com>
2024-08-19 19:24:07 +05:30
Sushant Mahajan c5e9154672
KAFKA-17342 Moved common coordinator code to separate module (#16883)
There is a lot of code in group-coordinator which is not share/consumer/classic group specific.

Since we are introducing a share-coordinator as part of KIP-932 (in a new module), it would make sense to get the common coordinator functionality into a separate common coordinator module so that share-coordinator need not depend on group-coordinator.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, David Jacot <djacot@confluent.io>, Andrew Schofield <aschofield@confluent.io>, Jeff Kim <jeff.kim@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-08-18 21:48:44 +08:00
TengYao Chi ad08ec600f
KAFKA-16682 Rewrite JaasTestUtils by Java (#16579)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-14 02:11:33 +08:00
TaiJuWu 5b9cbcf886
KAFKA-16689 Move LogValidatorTest to storage module (#16167)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-13 23:04:31 +08:00
PoAn Yang 49fc14f611
KAFKA-17273 Migrate BootstrapControllersIntegrationTest to use ClusterTestExtensions (#16822)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-11 19:48:13 +08:00
Colin Patrick McCabe e1b2adea07
KAFKA-17190: AssignmentsManager gets stuck retrying on deleted topics (#16672)
In MetadataVersion 3.7-IV2 and above, the broker's AssignmentsManager sends an RPC to the
controller informing it about which directory we have chosen to place each new replica on.
Unfortunately, the code does not check to see if the topic still exists in the MetadataImage before
sending the RPC. It will also retry infinitely. Therefore, after a topic is created and deleted in
rapid succession, we can get stuck including the now-defunct replica in our subsequent
AssignReplicasToDirsRequests forever.

In order to prevent this problem, the AssignmentsManager should check if a topic still exists (and
is still present on the broker in question) before sending the RPC. In order to prevent log spam,
we should not log any error messages until several minutes have gone past without success.
Finally, rather than creating a new EventQueue event for each assignment request, we should simply
modify a shared data structure and schedule a deferred event to send the accumulated RPCs. This
will improve efficiency.

Reviewers: Igor Soarez <i@soarez.me>, Ron Dagostino <rndgstn@gmail.com>
2024-08-10 12:31:45 +01:00
Colin Patrick McCabe 6a44fb154d
KAFKA-16523; kafka-metadata-quorum: support add-controller and remove-controller (#16774)
This PR adds support for add-controller and remove-controller in the kafka-metadata-quorum.sh
command-line tool. It also fixes some minor server-side bugs that blocked the tool from working.

In kafka-metadata-quorum.sh, the implementation of remove-controller is fairly straightforward. It
just takes some command-line flags and uses them to invoke AdminClient. The add-controller
implementation is a bit more complex because we have to look at the new controller's configuration
file. The parsing logic for the advertised.listeners and listeners server configurations that we
need was previously implemented in the :core module. However, the gradle module where
kafka-metadata-quorum.sh lives, :tools, cannot depend on :core. Therefore, I moved listener parsing
into SocketServerConfigs.listenerListToEndPoints. This will be a small step forward in our efforts
to move Kafka configuration out of :core.

I also made some minor changes in kafka-metadata-quorum.sh and Kafka-storage-tool.sh to handle
--help without displaying a backtrace on the screen, and give slightly better error messages on
stderr. Also, in DynamicVoter.toString, we now enclose the host in brackets if it contains a colon
(as IPV6 addresses can).

This PR fixes our handling of clusterId in addRaftVoter and removeRaftVoter, in two ways. Firstly,
it marks clusterId as nullable in the AddRaftVoterRequest.json and RemoveRaftVoterRequest.json
schemas, as it was always intended to be. Secondly, it allows AdminClient to optionally send
clusterId, by using AddRaftVoterOptions and RemoveRaftVoterOptions. We now also remember to
properly set timeoutMs in AddRaftVoterRequest. This PR adds unit tests for
KafkaAdminClient#addRaftVoter and KafkaAdminClient#removeRaftVoter, to make sure they are sending
the right things.

Finally, I fixed some minor server-side bugs that were blocking the handling of these RPCs.
Firstly, ApiKeys.ADD_RAFT_VOTER and ApiKeys.REMOVE_RAFT_VOTER are now marked as forwardable so that
forwarding from the broker to the active controller works correctly. Secondly,
org.apache.kafka.raft.KafkaNetworkChannel has now been updated to enable API_VERSIONS_REQUEST and
API_VERSIONS_RESPONSE.

Co-authored-by: Murali Basani muralidhar.basani@aiven.io
Reviewers: José Armando García Sancio <jsancio@apache.org>, Alyssa Huang <ahuang@confluent.io>
2024-08-08 15:54:12 -07:00
Mickael Maison 7c5d339d07
KAFKA-17227: Refactor compression code to only load codecs when used (#16782)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Josep Prat <josep.prat@aiven.io>
2024-08-06 11:01:21 +02:00
brenden20 0b78d8459c
MINOR: Remove ConsumerTestBuilder.java (#16691)
The purpose of this PR is to remove ConsumerTestBuilder.java since it is no longer needed. The following PRs have eliminated the use of ConsumerTestBuilder:
#14930
#16140
#16200
#16312

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-05 11:05:53 +08:00
Kuan-Po Tseng 84add30ea5
KAFKA-16154: Broker returns offset for LATEST_TIERED_TIMESTAMP (#16783)
This pr support EarliestLocalSpec LatestTierSpec in GetOffsetShell, and add integration tests.

Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, PoAn Yang <payang@apache.org>
2024-08-05 10:41:14 +08:00
Colin Patrick McCabe 02f541d4ea
KAFKA-16518: Implement KIP-853 flags for storage-tool.sh (#16669)
As part of KIP-853, storage-tool.sh now has two new flags: --standalone, and --initial-voters. This PR implements these two flags in storage-tool.sh.

There are currently two valid ways to format a cluster:

The pre-KIP-853 way, where you use a statically configured controller quorum. In this case, neither --standalone nor --initial-voters may be specified, and kraft.version must be set to 0.

The KIP-853 way, where one of --standalone and --initial-voters must be specified with the initial value of the dynamic controller quorum. In this case, kraft.version must be set to 1.

This PR moves the formatting logic out of StorageTool.scala and into Formatter.java. The tool file was never intended to get so huge, or to implement complex logic like generating metadata records. Those things should be done by code in the metadata or raft gradle modules. This is also useful for junit tests, which often need to do formatting. (The 'info' and 'random-uuid' commands remain in StorageTool.scala, for now.)

Reviewers: José Armando García Sancio <jsancio@apache.org>
2024-08-02 15:47:45 -07:00
Sebastien Viale f1ef7a5a9f
KAFKA-16448: Handle processing exceptions in punctuate (#16300)
This PR is part of KIP-1033 which aims to bring a ProcessingExceptionHandler to Kafka Streams in order to deal with exceptions that occur during processing.

This PR actually catches processing exceptions from punctuate.

Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: loicgreffier <loic.greffier@michelin.com>

Reviewers: Bruno Cadonna <bruno@confluent.io>, Matthias J. Sax <matthias@confluent.io>
2024-07-31 15:53:47 -07:00
Ken Huang fa682623b9
KAFKA-16666 Migrate OffsetMessageFormatter to tools module (#16689)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-07-31 15:18:14 +08:00
PoAn Yang 010ab19b72
KAFKA-16972 Move BrokerTopicMetrics to org.apache.kafka.storage.log.metrics (#16387)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-07-31 03:07:09 +08:00
Chris Egerton 61f61d6240
KAFKA-14569: Migrate Connect's integration test EmbeddedKafkaCluster from ZK to KRaft mode (#16599)
Reviewers: Omnia Ibrahim <o.g.h.ibrahim@gmail.com>, Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-07-29 10:43:55 -04:00
Kuan-Po Tseng 437b86cd20
KAFKA-17176 Move FileLock to server-common module (#16671)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-07-25 14:24:27 +08:00
Ken Huang ee68f3215f
KAFKA-16666 Migrate `TransactionLogMessageFormatter` to tools module (#16019)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-07-25 00:42:25 +08:00
Dmitry Werner 3e2de237be
KAFKA-17021 Migrate AclCommandTest to new test infra (#16500)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-07-25 00:12:47 +08:00
Ken Huang a012af5fb4
KAFKA-17149 Move ProducerStateManagerTest to storage module (#16645)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-07-23 20:08:08 +08:00
PoAn Yang defcbb51ee
KAFKA-17082 replace kafka.utils.LogCaptureAppender with org.apache.kafka.common.utils.LogCaptureAppender (#16601)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-07-21 18:22:05 +08:00
Abhijeet Kumar 24ed31739e
KAFKA-16853: Split RemoteLogManagerScheduledThreadPool (#16502)
As part of KIP-950, we want to split the RemoteLogManagerScheduledThreadPool into separate thread pools (one for copy and another for expiration). In this change, we are splitting it into three thread pools (one for copy, one for expiration, and another one for follower). We are reusing the same thread pool configuration for all three thread pools. We can introduce new user-facing configurations later.

Reviewers: Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Luke Chen <showuon@gmail.com>, Christo Lolov <lolovc@amazon.com>, Satish Duggana <satishd@apache.org>
2024-07-17 16:43:23 +05:30
Apoorv Mittal 0b6086ed88
KAFKA-16741: Add ShareGroupHeartbeat API support - 2/N (KIP-932) (#16573)
ShareGroupHeartbeat API support as defined in KIP-932. The heartbeat persists Group and Member information on __consumer_offsets topic.

The PR also moves some of the ShareGroupConfigs to GroupCoordinatorConfigs as they should only be used in group coordinator.


Reviewers: Andrew Schofield <aschofield@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>
2024-07-15 16:14:55 +05:30
Apoorv Mittal e0dcfa7b51
KAFKA-16741: Add share group classes for Heartbeat API (1/N) (KIP-932) (#16516)
Defined share group, member and sinmple assignor classes with API definition for Share Group Heartbeat and Describe API.

The ShareGroup and ShareGroupMember extends the common ModernGroup and ModernGroupMember respectively.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>
2024-07-04 20:31:47 +05:30
Abhinav Dixit 35baa0ac4f
KAFKA-17026: Implement updateCacheAndOffsets functionality on LSO movement (#16459)
Implemented the functionality which takes care of archiving the records when LSO moves past them. Implemented the following functions -

1. updateCacheAndOffsets - Updates the cached state, start and end offsets of the share partition as per the new log start offset. The method is called when the log start offset is moved for the share partition.
2. archiveAvailableRecordsOnLsoMovement - This function archives all the available records when they are behind the LSO.
3. archivePerOffsetBatchRecords - It archives all the available records in the per offset tracked batch passed to this function.
4. archiveCompleteBatch - It archives all the available records of the complete batch passed to this function.

Reviewers:  Andrew Schofield <aschofield@confluent.io>,Apoorv Mittal <apoorvmittal10@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
2024-07-03 16:01:44 +05:30
Chirag Wadhwa e55c28c60b
KAFKA-16750: Added acknowledge code in SharePartitionManager including unit tests (#16457)
About
This PR adds acknowledge code in SharePartitionManager. Internally, the record acknowledgements happen at the SharePartition level. SharePartitionManager identifies the SharePartitions and calls their acknowledge method to actually acknowledge the individual records

Testing
Added unit tests to cover the new functionality added in SharePartitionManagerTest

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
2024-07-02 22:25:14 +05:30
Kuan-Po (Cooper) Tseng 26b9163487
KAFKA-15623 (2/N) Migrate remaining tests in streams module to JUnit 5 (integration & internals) (#16360)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-20 09:24:44 +08:00
Omnia Ibrahim e99da2446c
KAFKA-15853: Move KafkaConfig.configDef out of core (#16116)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-06-14 17:26:00 +02:00
Kuan-Po (Cooper) Tseng 888a177603
KAFKA-12708 Rewrite org.apache.kafka.test.Microbenchmarks by JMH (#16231)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-14 16:47:34 +08:00
Apoorv Mittal 1565d41cd7
KAFKA-16752: Implemented acquire functionality for Fetch (KIP-932) (#16274)
The implementation for share-fetch next-fetch-offset in share partition and acquiring records from log.

The Next Fetch Offset (NFO) determines where the Share Partition should initiate the next data read from the Replica Manager. While it typically aligns with the last offset of the most recently returned batch, last offset + 1, there are exceptions. Messages marked available again due to release acknowledgements or lock timeouts can cause the NFO to shift.

The acquire method caches the batches as acquired in-memory and spawns a timer task for lock timeout.

Cache
Per-offset Metadata: Simple to implement but inefficient. Every offset requires in-memory storage and traversal, leading to high memory usage and processing overhead, especially for per-batch acknowledgements (mostly the way records would be acknowledged).

Per-Replica Fetch Batch: This approach aligns with the Replica Manager fetch batches. Since a full Replica Manager batch is retrieved whenever the requested offset falls within that batch's boundaries, a single Share Fetch request will likely receive an entire Replica Manager batch. However, there's a trade-off. Replica Manager batches are based on producer batching. If producers don't batch effectively, the in-flight metadata becomes heavily reliant on the producer's batching behavior.

For per-message acknowledgements, per-offset tracking will be necessary which again requires splitting in-flight batches based on state. Splitting bacthes is inefficient as it requires cache update wshich maintains sorted order. Therefore, we propose a hybrid approach:

Implemented a combination of option 2 (per-in-flight batch tracking) with option 1 (per-offset tracking). This aligns well with Replica Manager batching.

States shall be maintained per in-flight batch. If state inconsistencies arise within in-flight batches due to per-message acknowledgements, switch state tracking for the respective batch to option 1 (per-offset tracking).


Reviewers:  Andrew Schofield <aschofield@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>, Abhinav Dixit <144765188+adixitconfluent@users.noreply.github.com>
2024-06-14 10:31:56 +05:30
gongxuanzhang 596b945072
KAFKA-16643 Add ModifierOrder checkstyle rule (#15890)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-13 15:39:32 +08:00
Abhijeet Kumar 23fe71d579
KAFKA-15265: Integrate RLMQuotaManager for throttling copies to remote storage (#15820)
- Added the integration of the quota manager to throttle copy requests to the remote storage. Reference KIP-956
- Added unit-tests for the copy throttling logic.

Reviewers: Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>
2024-06-12 06:27:02 +05:30
Antoine Pourchet 8b6013f851
KAFKA-15045: (KIP-924 pt. 21) UUID to ProcessId migration (#16269)
This PR changes the assignment process to use ProcessId instead of UUID.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2024-06-11 12:24:44 -07:00
Antoine Pourchet 824900b97a
KAFKA-15045: (KIP-924 pt. 18) Better assignment testing (#16201)
Added more testing for the StickyTaskAssignor, which includes a large-scale test with rack aware enabled.

Also added a no-op change to StreamsAssignmentScaleTest.java to allow for rack aware optimization testing.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2024-06-08 19:48:43 -07:00
PoAn Yang 3d5d1504f7
KAFKA-16878 Remove powermock and easymock from code base (#16236)
Reviewers: TaiJuWu <tjwu1217@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-06-09 00:17:43 +08:00
Antoine Pourchet 0109a3f718
KAFKA-15045: (KIP-924 pt. 17) State store computation fixed (#16194)
Fixed the calculation of the store name list based on the subtopology being accessed.

Also added a new test to make sure this new functionality works as intended.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2024-06-05 13:09:37 -07:00
José Armando García Sancio 459da4795a
KAFKA-16525; Dynamic KRaft network manager and channel (#15986)
Allow KRaft replicas to send requests to any node (Node) not just the nodes configured in the
controller.quorum.voters property. This flexibility is needed so KRaft can implement the
controller.quorum.voters configuration, send request to the dynamically changing set of voters and
send request to the leader endpoint (Node) discovered through the KRaft RPCs (specially
BeginQuorumEpoch request and Fetch response).

This was achieved by changing the RequestManager API to accept Node instead of just the replica ID.
Internally, the request manager tracks connection state using the Node.idString method to match the
connection management used by NetworkClient.

The API for RequestManager is also changed so that the ConnectState class is not exposed in the
API. This allows the request manager to reclaim heap memory for any connection that is ready.

The NetworkChannel was updated to receive the endpoint information (Node) through the outbound raft
request (RaftRequent.Outbound). This makes the network channel more flexible as it doesn't need to
be configured with the list of all possible endpoints. RaftRequest.Outbound and
RaftResponse.Inbound were updated to include the remote node instead of just the remote id.

The follower state tracked by KRaft replicas was updated to include both the leader id and the
leader's endpoint (Node). In this comment the node value is computed from the set of voters. In
future commit this will be updated so that it is sent through KRaft RPCs. For example
BeginQuorumEpoch request and Fetch response.

Support for configuring controller.quorum.bootstrap.servers was added. This includes changes to
KafkaConfig, QuorumConfig, etc. All of the tests using QuorumTestHarness were changed to use the
controller.quorum.bootstrap.servers instead of the controller.quorum.voters for the broker
configuration. Finally, the node id for the bootstrap server will be decreasing negative numbers
starting with -2.

Reviewers: Jason Gustafson <jason@confluent.io>, Luke Chen <showuon@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
2024-06-03 14:24:48 -07:00
gongxuanzhang 342e69192f
KAFKA-10787 Add import ordering checkstyle rule and configure an automatic formatter (#16097)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-03 04:02:07 +08:00
Ken Huang 21caf6b123
KAFKA-16629 Add broker-related tests to ConfigCommandIntegrationTest (#15840)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-31 20:24:33 +08:00
Abhijeet Kumar bb7db87f98
KAFKA-15265: Add Remote Log Manager quota manager (#15625)
Added the implementation of the quota manager that will be used to throttle copy and fetch requests from the remote storage. Reference KIP-956

Reviewers: Luke Chen <showuon@gmail.com>, Kamal Chandraprakash <kchandraprakash@uber.com>, Jun Rao <junrao@gmail.com>
2024-05-30 09:06:49 -07:00
Dongnuo Lyu eefd114c4a
KAFKA-16832; LeaveGroup API for upgrading ConsumerGroup (#16057)
This patch implements the LeaveGroup API to the consumer groups that are in the mixed mode.

Reviewers: Jeff Kim <jeff.kim@confluent.io>, David Jacot <djacot@confluent.io>
2024-05-28 23:21:30 -07:00
Frederik Rouleau 4eb60b5104
KAFKA-16507 Add KeyDeserializationException and ValueDeserializationException with record content (#15691)
Implements KIP-1036.

Add raw ConsumerRecord data to RecordDeserialisationException to make DLQ implementation easier.

Reviewers: Kirk True <ktrue@confluent.io>, Andrew Schofield <aschofield@confluent.io>, Matthias J. Sax <matthias@confluent.io>
2024-05-28 14:56:47 -07:00
Greg Harris 11ad5e8bca
MINOR: Refactor Values class to fix checkstyle, add benchmark, optimize exceptions (#15469)
Signed-off-by: Greg Harris <greg.harris@aiven.io>
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-05-23 13:23:18 -07:00
PoAn Yang e93aae0664
KAFKA-16783: Migrate RemoteLogMetadataManagerTest to new test infra (#15983)
1. Replace TopicBasedRemoteLogMetadataManagerWrapperWithHarness with RemoteLogMetadataManagerTestUtils#builder in RemoteLogMetadataManagerTest.
2. Use ClusterTestExtention for RemoteLogMetadataManagerTest.

Signed-off-by: PoAn Yang <payang@apache.org>

Reviewers: Luke Chen <showuon@gmail.com>
2024-05-22 11:28:01 +08:00
Mickael Maison affe8da54c
KAFKA-7632: Support Compression Levels (KIP-390) (#15516)
Reviewers: Jun Rao <jun@confluent.io>,  Luke Chen <showuon@gmail.com>
Co-authored-by: Lee Dongjin <dongjin@apache.org>
2024-05-21 17:58:49 +02:00
Chia-Ping Tsai 2c51594607
MINOR: rewrite TopicBasedRemoteLogMetadataManagerTest by ClusterTestExtensions (#15917)
Reviewers: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Luke Chen <showuon@gmail.com>
2024-05-16 21:26:08 +08:00
Mickael Maison 0587a9af3d
MINOR: Various cleanups in clients tests (#15877)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>,  Lianet Magrans <lianetmr@gmail.com>
2024-05-14 10:19:00 +02:00
David Jacot c1e7aa77de
MINOR: Move `group-coordinator` import control to its own file (#15934)
Follow the pattern used by the other modules.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-14 00:05:32 -07:00