Commit Graph

698 Commits

Author SHA1 Message Date
Nikolay 6161fd0db2
KAFKA-14588 [2/N] ConfigCommandTest rewritten in java (#15873)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-13 19:45:28 +08:00
Jorge Esteban Quilcate Otoya a4c6cefd10
KAFKA-14226: Introduce FieldPath abstraction and nested path support for ExtractField SMT (#15379)
Reviewers: Chris Egerton <chrise@aiven.io>
2024-05-07 14:07:18 -04:00
TingIāu "Ting" Kì 0de3b7c40b
KAFKA-16593 Rewrite DeleteConsumerGroupsTest by ClusterTestExtensions (#15766)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-07 14:04:32 +08:00
David Jacot 0df340d64d
KAFKA-16470 kafka-dump-log --offsets-decoder should support new records (#15652)
When the consumer group protocol is used in a cluster, it is, at the moment, impossible to see all records stored in the __consumer_offsets topic with kafka-dump-log --offsets-decoder. It does not know how to handle all the new records.

This patch refactors the OffsetsMessageParser used internally by kafka-dump-log to use the RecordSerde used by the new group coordinator. It ensures that the tool is always in sync with the coordinator implementation. The patch also changes the format to using the toString'ed representations of the records instead of having custom logic to dump them. It ensures that all the information is always dumped. The downside of the latest is that inner byte arrays (e.g. assignment in the classic protocol) are no longer deserialized. Personally, I feel like that it is acceptable and it is actually better to stay as close as possible to the actual records in this tool. It also avoids issues like https://issues.apache.org/jira/browse/KAFKA-15603.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-07 08:49:31 +08:00
José Armando García Sancio bfe81d6229
KAFKA-16207; KRaft's internal log listener to update voter set (#15671)
Adds support for the KafkaRaftClient to read the control records KRaftVersionRecord and VotersRecord in the snapshot and log. As the control records in the KRaft partition are read, the replica's known set of voters are updated. This change also contains the necessary changes to include the control records when a snapshot is generated by the KRaft state machine.

It is important to note that this commit changes the code and the in-memory state to track the sets of voters but it doesn't change any data that is externally exposed. It doesn't change the RPCs, data stored on disk or configuration.

When the KRaft replica starts the PartitionListener reads the latest snapshot and then log segments up to the LEO, updating the in-memory state as it reads KRaftVersionRecord and VotersRecord. When the replica (leader and follower) appends to the log, the PartitionListener catches up to the new LEO. When the replica truncates the log because of a diverging epoch, the PartitionListener also truncates the in-memory state to the new LEO. When the state machine generate a new snapshot the PartitionListener trims any prefix entries that are not needed. This is all done to minimize the amount of data tracked in-memory and to make sure that it matches the state on disk.

To implement the functionality described above this commit also makes the following changes:

Adds control records for KRaftVersionRecord and VotersRecord. KRaftVersionRecord describes the finalized kraft.version supported by all of the replicas. VotersRecords describes the set of voters at a specific offset.

Changes Kafka's feature version to support 0 as the smallest valid value. This is needed because the default value for kraft.version is 0.

Refactors FileRawSnapshotWriter so that it doesn't directly call the onSnapshotFrozen callback. It adds NotifyingRawSnapshotWriter for calling such callbacks. This reorganization is needed because in this change both the KafkaMetadataLog and the KafkaRaftClient need to react to snapshots getting frozen.

Cleans up KafkaRaftClient's initialization. Removes initialize from RaftClient - this is an implementation detail that doesn't need to be exposed in the interface. Removes RaftConfig.AddressSpec and simplifies the bootstrapping of the static voter's address. The bootstrapping of the address is delayed because of tests. We should be able to simplify this further in future commits.

Update the DumpLogSegment CLI to support the new control records KRaftVersionRecord and VotersRecord.

Fix the RecordsSnapshotReader implementations so that the iterator includes control records. RecordsIterator is extended to support reading the new control records.
Improve the BatchAccumulator implementation to allow multiple control records in one control batch. This is needed so that KRaft can make sure that VotersRecord is included in the same batch as the control record (KRaftVersionRecord) that upgrades the kraft.version to 1.

Add a History interface and default implementation TreeMapHistory. This is used to track all of the sets of voters between the latest snapshot and the LEO. This is needed so that KafkaRaftClient can query for the latest set of voters and so that KafkaRaftClient can include the correct set of voters when the state machine generates a new snapshot at a given offset.

Add a builder pattern for RecordsSnapshotWriter. The new builder pattern also implements including the KRaftVersionRecord and VotersRecord control records in the snapshot as necessary. A KRaftVersionRecord should be appended if the kraft.version is greater than 0 at the snapshot's offset. Similarly, a VotersRecord should be appended to the snapshot with the latest value up to the snapshot's offset.

Reviewers: Jason Gustafson <jason@confluent.io>
2024-05-04 12:43:16 -07:00
Nikolay cdc4caa578
KAFKA-14588 UserScramCredentialsCommandTest rewritten in Java (#15832)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Igor Soarez <soarez@apple.com>
2024-05-02 10:35:10 +01:00
Nikolay 81c24d6bf8
KAFKA-15588 ConfigCommandIntegrationTest rewritten in java (#15645)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-30 01:46:08 +08:00
Omnia Ibrahim d88c15fc3e
KAFKA-15853 Move KRAFT configs out of KafkaConfig (#15775)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-27 07:02:31 +08:00
Omnia Ibrahim cfe5ab5cf2
KAFKA-15853 Move quota configs into server-common package (#15774)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-24 13:05:18 +08:00
Cheng-Kai, Zhang b6e70e9a54
MINOR: Add test for PartitionMetadataFile (#15714)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-24 13:01:35 +08:00
Omnia Ibrahim 1b301b3020
KAFKA-15853 Move socket configs into org.apache.kafka.network.SocketServerConfigs (#15772)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-23 17:39:36 +08:00
Omnia Ibrahim ecb2dd4cdc
KAFKA-15853 Move KafkaConfig log properties and docs out of core (#15569)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Nikolay <nizhikov@apache.org>, Federico Valeri <fvaleri@redhat.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-20 04:14:23 +08:00
Omnia Ibrahim 363f4d2847
KAFKA-15853 Move consumer group and group coordinator configs out of core (#15684)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-17 20:41:22 +08:00
Mickael Maison fccd7fec66
MINOR: Various cleanups in clients (#15705)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-16 15:02:03 +02:00
Omnia Ibrahim 8c0458861c
KAFKA-15853 Move KafkaConfig Replication properties and docs out of … (#15575)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-16 15:28:35 +08:00
Omnia Ibrahim 61baa7ac6b
KAFKA-15853 Move transactions configs out of core (#15670)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-13 00:29:51 +08:00
Nikolay d8673b26bf
KAFKA-15899 [1/2] Move kafka.security package from core to server module (#15572)
1) This PR moves kafka.security classes from core to server module.
2) AclAuthorizer not moved, because it has heavy dependencies on core classes that not rewrited from scala at the moment.
3) AclAuthorizer will be deleted as part of ZK removal

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-30 11:54:22 +08:00
Greg Harris bf5e04e416
KAFKA-16349: Prevent race conditions in Exit class from stopping test JVM (#15484)
Signed-off-by: Greg Harris <greg.harris@aiven.io>
Reviewers: Chris Egerton <chrise@aiven.io>
2024-03-28 20:07:42 -07:00
Nikolay 6f38fe5e0a
KAFKA-14588 ZK configuration moved to ZkConfig (#15075)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-27 22:37:01 +08:00
Hector Geraldino 178761eb36
KAFKA-14683 Cleanup WorkerSinkTaskTest (#15506)
1) Rename WorkerSinkTaskMockitoTest back to WorkerSinkTaskTest
2) Tidy up the code a bit
3) rewrite "fail" by "assertThrow"

Reviewers: Omnia Ibrahim <o.g.h.ibrahim@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-03-15 03:50:57 +08:00
Daan Gerits b9a5b4a805
KAFKA-10892: Shared Readonly State Stores ( revisited ) (#12742)
Implements KIP-813.

Reviewers: Matthias J. Sax <matthias@confluent.io>, Walker Carlson <wcarlson@confluent.io>
2024-03-08 10:57:56 -08:00
testn 80def43a34
MINOR: Reduce memory allocation in ClientTelemetryReporter (#15402)
Reviewers: Divij Vaidya <diviv@amazon.com>
2024-03-08 17:43:44 +01:00
Hector Geraldino 62998b7264
KAFKA-14683: Migrate WorkerSinkTaskTest to Mockito (3/3) (#15316)
Reviewers: Greg Harris <greg.harris@aiven.io>
2024-03-06 10:31:33 -08:00
Nikolay f6198bc075
KAFKA-14589 [3/4] Tests of ConsoleGroupCommand rewritten in java (#15365)
Is contains some of ConsoleGroupCommand tests rewritten in java.
Intention of separate PR is to reduce changes and simplify review.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-06 17:13:39 +08:00
Nikolay eea369af94
KAFKA-14588 Log cleaner configuration move to CleanerConfig (#15387)
In order to move ConfigCommand to tools we must move all it's dependencies which includes KafkaConfig and other core classes to java. This PR moves log cleaner configuration to CleanerConfig class of storage module.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-05 18:11:56 +08:00
Ritika Reddy 96c68096a2
KAFKA-15462: Add Group Type Filter for List Group to the Admin Client (#15150)
In KIP-848, we introduce the notion of Group Types based on the protocol type that the members in the consumer group use. As of now we support two types of groups:
* Classic : Members use the classic consumer group protocol ( existing one )
* Consumer : Members use the consumer group protocol introduced in KIP-848.
Currently List Groups allows users to list all the consumer groups available. KIP-518 introduced filtering the consumer groups by the state that they are in. We now want to allow users to filter consumer groups by type.

This patch includes the changes to the admin client and related files. It also includes changes to parameterize the tests to include permutations of the old GC and the new GC with the different protocol types.

Reviewers: David Jacot <djacot@confluent.io>
2024-02-29 00:38:42 -08:00
David Jacot d24abe0ede
MINOR: Refactor GroupMetadataManagerTest (#15348)
`GroupMetadataManagerTest` class got a little under control. We have too many things defined in it. As a first steps, this patch extracts all the inner classes. It also extracts all the helper methods. However, the logic is not changed at all.

Reviewers: Omnia Ibrahim <o.g.h.ibrahim@gmail.com>, Justine Olshan <jolshan@confluent.io>
2024-02-13 23:29:29 -08:00
Mickael Maison 0bf830fc9c
KAFKA-14576: Move ConsoleConsumer to tools (#15274)
Reviewers: Josep Prat <josep.prat@aiven.io>, Omnia Ibrahim <o.g.h.ibrahim@gmail.com>
2024-02-13 19:24:07 +01:00
Nikolay 88c5543ccf
KAFKA-14589: [1/3] Tests of ConsoleGroupCommand rewritten in java (#15256)
This PR is part of #14471
Is contains some of ConsoleGroupCommand tests rewritten in java.
Intention of separate PR is to reduce changes and simplify review.

Reviewers: Luke Chen <showuon@gmail.com>
2024-02-13 11:02:36 +08:00
ghostspiders 5cfcc52fb3
KAFKA-16239: Clean up references to non-existent IntegrationTestHelper (#15352)
Co-authored-by: ghostspiders <yufeng.gao@seres.cn>

Reviewers: Divij Vaidya <diviv@amazon.com>
2024-02-12 13:27:47 +01:00
Chris Egerton 4f0a405908
KAFKA-15575: Begin enforcing 'tasks.max' property for connectors (#15180)
Reviewers: Ashwin Pankaj <apankaj@confluent.io>, Greg Harris <greg.harris@aiven.io>
2024-02-01 11:33:04 -05:00
Omnia Ibrahim 127fe7d276
KAFKA-15853: Move AuthorizerUtils and its dependencies to server module (#15167)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-01-31 15:38:14 +01:00
Mickael Maison 3e9ef70853
KAFKA-15853: Move PasswordEncoder to server-common (#15246)
Reviewers: Luke Chen <showuon@gmail.com>, Omnia Ibrahim <o.g.h.ibrahim@gmail.com>
2024-01-30 19:08:50 +01:00
Nikolay 13c0c5ee97
KAFKA-14589 ConsumerGroupServiceTest rewritten in java (#15248)
This PR is part of #14471
Is contains single test rewritten in java.
Intention of separate PR is to reduce changes and simplify review.

Reviewers: Justine Olshan <jolshan@confluent.io>
2024-01-26 10:32:48 -08:00
Lianet Magrans 44272eaa77
KAFKA-16032: Fixes for commit/fetch error handling (#15202)
This includes multiple fixes for offsets commit/fetch error handling:
* ensure the right exceptions are thrown for each expected error
* ensure KafkaException is thrown for all unexpected errors
* properly handle disconnection exceptions (added for fetch, fixed for commit)

Reviewers: David Jacot <djacot@confluent.io>
2024-01-26 05:42:54 -08:00
Calvin Liu 7e5ef9b509
KAFKA-15585: Implement DescribeTopicPartitions RPC on broker (#14612)
This patch implements the new DescribeTopicPartitions RPC as defined in KIP-966 (ELR). Additionally, this patch adds a broker config "max.request.partition.size.limit" which limits the number of partitions returned by the new RPC.

Reviewers: Artem Livshits <alivshits@confluent.io>, Jason Gustafson <jason@confluent.io>, David Arthur <mumrah@gmail.com>
2024-01-24 15:16:09 -05:00
Apoorv Mittal 208f9e7765
KAFKA-15813: Evict client instances from cache (KIP-714) (#15234)
KIP-714 requires client instance cache in broker which should also have a time-based eviction policy where client instances which are not actively sending metrics should be evicted. KIP mentions This client instance specific state is maintained in broker memory up to MAX(60*1000, PushIntervalMs * 3) milliseconds.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Jun Rao <junrao@gmail.com>
2024-01-23 15:06:02 -08:00
Greg Harris d1d6b5096f
KAFKA-16166: Generify RetryWithToleranceOperator, ErrorReporter, and WorkerTask (#15233)
Signed-off-by: Greg Harris <greg.harris@aiven.io>
Reviewers: Chris Egerton <chrise@aiven.io>
2024-01-22 16:56:52 -08:00
Nikolay ff25c350a7
KAFKA-14589 ConsumerGroupCommand options and case classes rewritten (#14856)
This PR is part of #14471
It contains ConsumerGroupCommandOptions and case classes used by ConsumerGroupCommand rewritten in java.

The goal of PR is to reduce main PR size.

Co-authored-by: Taras Ledkov <tledkov@apache.org>

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Taras Ledkov <tledkov@apache.org>, Justine Olshan <jolshan@confluent.io>
2024-01-22 16:30:34 -08:00
Omnia Ibrahim 62ce551826
KAFKA-15853: Move KafkaConfig.Defaults to server module (#15158)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ismael Juma <ismael@juma.me.uk>
, David Jacot <djacot@confluent.io>, Nikolay <NIzhikov@gmail.com>
2024-01-22 15:29:11 +01:00
Apoorv Mittal 2df8c1ca3d
KAFKA-15807: Added support for compression of metrics (KIP-714) (#15148)
Part of KIP-714.

Adds support for compression/decompression of metrics.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Philip Nee <pnee@confluent.io>, Matthias J. Sax <matthias@confluent.io>
2024-01-17 13:49:57 -08:00
David Jacot a8203f9c7a
KAFKA-14505; [4/N] Wire transaction verification (#15142)
This patch wires the transaction verification in the new group coordinator. It basically calls the verification path before scheduling the write operation. If the verification fails, the error is returned to the caller.

Note that the patch uses `appendForGroup`. I suppose that we will move away from using it when https://github.com/apache/kafka/pull/15087 is merged.

Reviewers: Justine Olshan <jolshan@confluent.io>
2024-01-11 04:58:57 -08:00
Chris Egerton dbf00bcf45
KAFKA-16093: Fix spurious REST-related warnings on Connect startup (#15149)
Reviewers: Sagar Rao <sagarmeansocean@gmail.com>, Greg Harris <greg.harris@aiven.io>
2024-01-10 09:03:23 -05:00
Jason Gustafson 599e22b842
MINOR: Move Raft io thread implementation to Java (#15119)
This patch moves the `RaftIOThread` implementation into Java. I changed the name to `KafkaRaftClientDriver` since the main thing it does is drive the calls to `poll()`. There shouldn't be any changes to the logic.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2024-01-05 09:27:36 -08:00
Nick Telford 5bc3aa4280
KAFKA-14412: Decouple RocksDB access from CF (#15105)
To support future use-cases that use different strategies for accessing
RocksDB, we need to de-couple the RocksDB access strategy from the
Column Family access strategy.

To do this, we now have two separate accessors:

  * `DBAccessor`: dictates how we access RocksDB. Currently only one
    strategy is supported: `DirectDBAccessor`, which access RocksDB
    directly, via the `RocksDB` class for all operations. In the future, a
    `BatchedDBAccessor` will be added, which enables transactions via
    `WriteBatch`.
  * `ColumnFamilyAccessor`: maps StateStore operations to operations on
    one or more column families. This is a rename of the old
    `RocksDBDBAccessor`.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2024-01-04 11:42:30 +01:00
Nikolay 45bd19f2ef
KAFKA-14588: Move ConfigType to server-common (#14867)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2023-12-22 18:35:27 +01:00
Bruno Cadonna 19727f8d51
KAFKA-16017: Checkpoint restored offsets instead of written offsets (#15044)
Kafka Streams checkpoints the wrong offset when a task is closed during
restoration. If under exactly-once processing guarantees a
TaskCorruptedException happens, the affected task is closed dirty, its
state content is wiped out and the task is re-initialized. If during
the following restoration the task is closed cleanly, the task writes
the offsets that it stores in its record collector to the checkpoint
file. Those offsets are the offsets that the task wrote to the changelog
topics. In other words, the task writes the end offsets of its changelog
topics to the checkpoint file. Consequently, when the task is
initialized again on the same Streams client, the checkpoint file is
read and the task assumes it is fully restored although the records
between the last offsets the task restored before closing clean and
the end offset of the changelog topics are missing locally.

The fix is to clear the offsets in the record collector on close.

Reviewer: Lucas Brutschy <lbrutschy@confluent.io>
2023-12-21 10:15:04 +01:00
Kirk True 9dc9040f33
KAFKA-15276: Implement event plumbing for ConsumerRebalanceListener callbacks (#14640)
This patch adds the logic for coordinating the invocation of the `ConsumerRebalanceListener` callback invocations between the background thread (in `MembershipManagerImpl`) and the application thread (`AsyncKafkaConsumer`) and back again. It allowed us to enable more tests from `PlaintextConsumerTest` to exercise the code herein.

Reviewers: David Jacot <djacot@confluent.io>
2023-12-15 00:42:31 -08:00
Michael Westerby ccb36fe0c7
KAFKA-15358: Added QueuedReplicaToDirAssignments metric (#14942)
Reviewers: Igor Soarez <i@soarez.me>, Ron Dagostino <rdagostino@confluent.io>
2023-12-12 03:17:00 -05:00
Chris Egerton 2a5fbf2882
KAFKA-15563: Provide informative error messages when Connect REST requests time out (#14562)
Reviewers: Greg Harris <greg.harris@aiven.io>
2023-12-11 16:48:16 -05:00
Colin Patrick McCabe 32fdb8d173
KAFKA-15956: MetadataShell must take the log directory lock when reading (#14899)
MetadataShell should take an advisory lock on the .lock file of the directory it is reading from.
Add an integration test of this functionality in MetadataShellIntegrationTest.java.

Note: in build.gradle, I had to add some dependencies on server-common's test files in order to use
MockFaultHandler, etc.

MetadataBatchLoader.java: fix a case where a log message was incorrect.  The intention was to print
the number equivalent to (offset + index).  Instead it was printing the offset, followed by the
index. So if the offset was 100 and the index was 1, 1001 would be printed rather than 101.

Co-authored-by: Igor Soarez <i@soarez.me>
Reviewers: David Arthur <mumrah@gmail.com>, José Armando García Sancio <jsancio@apache.org>
2023-12-10 19:18:34 -08:00
Igor Soarez c515bf51f8 KAFKA-15426: Process and persist directory assignments
Handle AssignReplicasToDirs requests, persist metadata changes
with new directory assignments and possible leader elections.

Reviewers: Proven Provenzano <pprovenzano@confluent.io>, Ron Dagostino <rndgstn@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
2023-12-07 11:44:45 -08:00
Matthias J. Sax 7dabd27f8d
KAFKA-15662: Add support for clientInstanceIds in Kafka Stream (#14922)
Part of KIP-714.

Adds support to expose main consumer client instance id.

Reviewers: Walker Carlson <wcarlson@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>
2023-12-07 10:39:39 -08:00
David Jacot 522c2864cd
KAFKA-14505; [2/N] Implement TxnOffsetCommit API (#14845)
This patch implements the TxnOffsetCommit API. When a transactional offset commit is received, it is stored in the pending transactional offsets structure and waits there until the transaction is committed or aborted. Note that the handling of the transaction completion is not implemented in this patch.

Reviewers: Justine Olshan <jolshan@confluent.io>
2023-12-07 02:51:22 -08:00
Alieh Saeedi 9658942366
KAFKA-15347: add support for 'single key multi timestamp' IQs with versioned state stores (KIP-968) (#14626)
Implements KIP-968.

Add new query type MultiVersionedKeyQuery for IQv2 supported by versioned state stores.
2023-12-06 07:56:12 -08:00
Apoorv Mittal 2b99d0e450
KAFKA-15901: Client changes for registering telemetry and API calls (KIP-714) (#14843)
The PR adds changes for the client APIs to register ClientTelemetryReporter, if enabled, and periodically report client metrics. The changes include front facing API changes with NetworkCLient issuing telemetry APIs.

The PR build is dependent on: #14620, #14724

Reviewers: Philip Nee <pnee@confluent.io>, Andrew Schofield <aschofield@confluent.io>, Kirk True <ktrue@confluent.io>, Matthias J. Sax <matthias@confluent.io>, Walker Carlson <wcarlson@apache.org>
2023-12-05 11:50:33 -06:00
Nikolay 783698c525
KAFKA-15645: Move ReplicationQuotasTestRig to tools module (#14588)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Justine Olshan <jolshan@confluent.io>, Taras Ledkov <tledkov@apache.org>
2023-12-05 10:03:33 +01:00
Apoorv Mittal 7a6d2664cd
KAFKA-15663, KAFKA-15794: Telemetry reporter and request handling (KIP-714) (#14909)
Part of KIP-714.

Implements ClientTelemetryReporter which manages the lifecycle for client metrics collection. The reporter also defines TelemetrySender which will be used by Network clients to send API calls to broker.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Philip Nee <pnee@confluent.io>, Matthias J. Sax <matthias@confluent.io>
2023-12-04 11:44:56 -08:00
Andrew Schofield 1750d735cd
KAFKA-15842: Correct handling of KafkaConsumer.committed for new consumer (#14859)
This PR fixes some details of the interface to KafkaConsumer.committed which were different between the existing consumer and the new consumer.

Adds a unit test that validates the behaviour is the same for both consumer implementations.

Reviewers: Kirk True <ktrue@confluent.io>, Bruno Cadonna <cadonna@apache.org>
2023-12-01 14:37:21 +01:00
Hanyu Zheng f1cd11dcc5
KAFKA-15629: Proposal to introduce IQv2 Query Types: TimestampedKeyQuery and TimestampedRangeQuery (#14570)
Implements KIP-992.

Adds TimestampedKeyQuery and TimestampedRangeQuery (IQv2) for ts-ks-store, plus changes semantics of existing KeyQuery and RangeQuery if issues against a ts-kv-store, now unwrapping value-and-timestamp and only returning the plain value.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2023-11-30 12:14:23 -08:00
Apoorv Mittal f1819f4480
KAFKA-15778 & KAFKA-15779: Implement metrics manager (KIP-714) (#14699)
The PR provide implementation for client metrics manager along with other classes. Manager is responsible to support 3 operations:

UpdateSubscription - From kafka-configs.sh and reload from metadata cache.
Process Get Telemetry Request - From KafkaApis.scala
Process Push Telemetry Request - From KafkaApis.scala
Manager maintains an in-memory cache to keep track of client instances against their instance id.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Jun Rao <junrao@gmail.com>
2023-11-29 09:20:07 -08:00
Apoorv Mittal 009b57d870
KAFKA-15618: Kafka metrics collector and supporting classes (KIP-714) (#14620)
The PR outlines classes to collect metrics for client by KafkaMetricsCollector implementation. The MetricsCollector defines mechanism to collect client metrics in sum and gauge metrics format. This requires to define cumulative and delta telemetry metrics while collecting raw metrics.

Singl point metric class helps creating OTLP format Metric object wrapped over Single point metric class itself.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Xavier Léauté <xavier@confluent.io>, Philip Nee <pnee@confluent.io>, Matthias J. Sax <matthias@confluent.io>
2023-11-28 22:07:22 -08:00
Apoorv Mittal 38f2faf83f
KAFKA-15681: Add support of client-metrics in kafka-configs.sh (KIP-714) (#14632)
The PR adds support of alter/describe configs for client-metrics as defined in KIP-714

Reviewers: Andrew Schofield <aschofield@confluent.io>, Jun Rao <junrao@gmail.com>
2023-11-28 09:24:25 -08:00
Ritika Reddy 55017a4f68
KAFKA-15484: General Rack Aware Assignor (#14481)
This patch adds the second part of the Uniform Assignor, used when the subscriptions of each member in a consumer group are different.

Reviewers: Jeff Kim <jeff.kim@confluent.io>, David Jacot <djacot@confluent.io>
2023-11-23 01:18:50 -08:00
Jeff Kim 07fee62afe
KAFKA-14519; [2/N] New coordinator metrics (#14387)
This patch copy over existing metrics and add new consumer group metrics to the new GroupCoordinatorService.

Now that each coordinator is responsible for a topic partition, this patch introduces a GroupCoordinatorMetrics that records gauges for global metrics such as the number of generic groups in PreparingRebalance state, etc. For GroupCoordinatorShard specific metrics, GroupCoordinatorMetrics will activate new GroupCoordinatorMetricsShards that will be responsible for incrementing/decrementing TimelineLong objects and then aggregate the total amount across all shards.

As the CoordinatorRuntime/CoordinatorShard does not care about group metadata, we have introduced a CoordinatorMetrics.java/CoordinatorMetricsShard.java so that in the future transaction coordinator metrics can also be onboarded in a similar fashion.

Main files to look at:

GroupCoordinatorMetrics.java
GroupCoordinatorMetricsShard.java
CoordinatorMetrics.java
CoordinatorMetricsShard.java
CoordinatorRuntime.java
Metrics to add after #14408 is merged:

offset deletions sensor (OffsetDeletions); Meter(offset-deletion-rate, offset-deletion-count)
Metrics to add after https://issues.apache.org/jira/browse/KAFKA-14987 is merged:

offset expired sensor (OffsetExpired); Meter(offset-expiration-rate, offset-expiration-count)

Reviewers: Justine Olshan <jolshan@confluent.io>
2023-11-20 21:38:50 -08:00
Ismael Juma df78204e05
KAFKA-15854: Move Java classes from `kafka.server` to the `server` module (#14796)
We only move Java classes that have minimal or no dependencies on Scala classes in this PR.

Details:
* Configured `server` module in build files.
* Changed `ControllerRequestCompletionHandler` to be an interface since it has no implementations.
* Cleaned up various import control files.
* Minor build clean-ups for `server-common`.
* Disabled `testAssignmentAggregation` when executed with Java 8, this is an existing issue (see #14794).

For broader context on this change, please check:
* KAFKA-15852: Move server code from `core` to `server` module

Reviewers: Divij Vaidya <diviv@amazon.com>
2023-11-19 22:09:19 -08:00
Igor Soarez a03a71d7b5 KAFKA-15357: Aggregate and propagate assignments
A new AssignmentsManager accumulates, batches, and sends KIP-858
assignment events to the Controller. Assignments are sent via
AssignReplicasToDirs requests.

Move QuorumTestHarness.formatDirectories into TestUtils so it can be
used in other test contexts.

Fix a bug in ControllerRegistration.java where the wrong version of the
record was being generated in ControllerRegistration.toRecord.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, Proven Provenzano <pprovenzano@confluent.io>, Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>
2023-11-16 16:19:49 -08:00
Yash Mayya 1bc4de75a2
KAFKA-15470: Allow creating connectors in a stopped state (#14704)
Reviewers: Chris Egerton <chrise@aiven.io>
2023-11-15 11:37:50 +05:30
Colin Patrick McCabe 7060c08d6f
MINOR: Rewrite the meta.properties handling code in Java and fix some issues #14628 (#14628)
meta.properties files are used by Kafka to identify log directories within the filesystem.
Previously, the code for handling them was in BrokerMetadataCheckpoint.scala. This PR rewrites the
code for handling them as Java and moves it to the apache.kafka.metadata.properties namespace. It
also gets rid of the separate types for v0 and v1 meta.properties objects. Having separate types
wasn't so bad back when we had a strict rule that zk clusters used v0 and kraft clusters used v1.
But ZK migration has blurred the lines. Now, a zk cluster may have either v0 or v1, if it is
migrating, and a kraft cluster may have either v0 or v1, at any time.

The new code distinguishes between an individual meta.properties file, which is represented by
MetaProperties, and a collection of meta.properties files, which is represented by
MetaPropertiesEnsemble. It is useful to have this distinction, because in JBOD mode, even if some
log directories are inaccessible, we can still use the ensemble to extract needed information like
the cluster ID. (Of course, even when not in JBOD mode, KRaft servers have always been able to
configure a metadata log directory separate from the main log directory.)

Since we recently added a unique directory.id to each meta.properties file, the previous convention
of passing a "canonical" MetaProperties object for the cluster around to various places in the code
needs to be revisited. After all, we can no longer assume all of the meta.properties files are the
same. This PR fixes these parts of the code. For example, it fixes the constructors of
ControllerApis and RaftManager to just take a cluster ID, rather than a MetaProperties object. It
fixes some other parts of the code, like the constructor of SharedServer, to take a
MetaPropertiesEnsemble object.

Another goal of this PR was to centralize meta.properties validation a bit more and make it
unit-testable. For this purpose, the PR adds MetaPropertiesEnsemble.verify, and a few other
verification methods. These enforce invariants like "the metadata directory must be readable," and
so on.

Reviewers: Igor Soarez <soarez@apple.com>, David Arthur <mumrah@gmail.com>, Divij Vaidya <diviv@amazon.com>, Proven Provenzano <pprovenzano@confluent.io>
2023-11-09 09:32:35 -08:00
Calvin Liu 505e5b3eaa
KAFKA-15584: Leader election with ELR (#14593)
The patch includes the following changes as part of KIP-966

* Allow ISR shrink to empty
* Allow leader election with ELR members
* Allow electing the last known leader

Reviewers: Artem Livshits <alivshits@confluent.io>, David Arthur <mumrah@gmail.com>
2023-11-06 17:21:51 -05:00
Igor Soarez 0390d5b1a2
KAFKA-15355: Message schema changes (#14290)
Reviewers: Christo Lolov <lolovc@amazon.com>, Colin P. McCabe <cmccabe@apache.org>, Proven Provenzano <pprovenzano@confluent.io>, Ron Dagostino <rdagostino@confluent.io>
2023-11-02 09:46:05 -04:00
Calvin Liu 8f8ad6db38
KAFKA-15582: Move the clean shutdown file to the storage package (#14603)
A follow-up change to move the clean shutdown file to the storage package.

Reviewers: Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>
2023-10-30 16:27:40 -07:00
Apoorv Mittal ad2677bb7b
KAFKA-15614: Define interfaces and classes for client telemetry (#14575)
This PR for KIP-714 - KAFKA-1564 lays out interfaces and classes for capturing client telemetry metrics.

Below image defines interaction of different classes among them interfaces have been included in the PR.

Reviewers: Walker Carlson <wcarlson@apache.org>, Matthias J. Sax <matthias@confluent.io>, Andrew Schofield <andrew_schofield@uk.ibm.com>, Kirk True <ktrue@confluent.io>, Philip Nee <pnee@confluent.io>, Jun Rao <junrao@gmail.com>,
2023-10-26 15:06:38 -05:00
Kirk True 2b233bfa5f
KAFKA-14274 [6, 7]: Introduction of fetch request manager (#14406)
Changes:

1. Introduces FetchRequestManager that implements the RequestManager
   API for fetching messages from brokers. Unlike Fetcher, record
   decompression and deserialization is performed on the application
   thread inside CompletedFetch.
2. Restructured the code so that objects owned by the background thread
   are not instantiated until the background thread runs (via Supplier)
   to ensure that there are no references available to the
   application thread.
3. Ensuring resources are properly using Closeable and using
   IdempotentCloser to ensure they're only closed once.
4. Introduces ConsumerTestBuilder to reduce a lot of inconsistency in
   the way the objects were built up for tests.

Reviewers: Philip Nee <pnee@confluent.io>, Lianet Magrans <lianetmr@gmail.com>, Jun Rao<junrao@gmail.com>
2023-10-24 13:03:05 -07:00
Chris Egerton 091eb9b349
KAFKA-15428: Cluster-wide dynamic log adjustments for Connect (#14538)
Reviewers: Greg Harris <greg.harris@aiven.io>, Yang Yang <yayang@uber.com>, Yash Mayya <yash.mayya@gmail.com>
2023-10-20 09:52:37 -04:00
Calvin Liu af747fbfed
KAFKA-15581: Introduce ELR (#14312)
This patch introduces preliminary changes for Eligible Leader Replicas (KIP-966)

* New MetadataVersion 16 (3.7-IV1)
* New record versions for PartitionRecord and PartitionChangeRecord
* New tagged fields on PartitionRecord and PartitionChangeRecord
* New static config "eligible.leader.replicas.enable" to gate the whole feature

Reviewers: Artem Livshits <alivshits@confluent.io>, David Arthur <mumrah@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
2023-10-19 14:05:15 -04:00
Calvin Liu 14029e2ddd
KAFKA-15582: Identify clean shutdown broker (#14465)
The PR includes:

* Added a new class of CleanShutdownFile which helps write and read from a clean shutdown file.
* Updated the BrokerRegistration API.
* Client side handling for the broker epoch.
* Minimum work on the controller side.

Reviewers: Jun Rao <junrao@gmail.com>
2023-10-19 10:25:23 -07:00
Apoorv Mittal 36abc8dcea
KAFKA-15604: Telemetry API request and response schemas and classes (KIP-714) (#14554)
Initial PR for [KIP-714](https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability) - [KAFKA-15601](https://issues.apache.org/jira/browse/KAFKA-15601).

This PR defines json request and response schemas for the new Telemetry APIs and implements the corresponding java classes.

Reviewers: 
Andrew Schofield <andrew_schofield@uk.ibm.com>, Kirk True <ktrue@confluent.io>, Matthias J. Sax <matthias@confluent.io>, Walker Carlson <wcarlson@apache.org>
2023-10-19 10:55:21 -05:00
Jeff Kim abee8f711c
KAFKA-14519; [1/N] Implement coordinator runtime metrics (#14417)
Implements the following metrics:

kafka.server:type=group-coordinator-metrics,name=num-partitions,state=loading
kafka.server:type=group-coordinator-metrics,name=num-partitions,state=active
kafka.server:type=group-coordinator-metrics,name=num-partitions,state=failed
kafka.server:type=group-coordinator-metrics,name=event-queue-size
kafka.server:type=group-coordinator-metrics,name=partition-load-time-max
kafka.server:type=group-coordinator-metrics,name=partition-load-time-avg
kafka.server:type=group-coordinator-metrics,name=thread-idle-ratio-min
kafka.server:type=group-coordinator-metrics,name=thread-idle-ratio-avg
The PR makes these metrics generic so that in the future the transaction coordinator runtime can implement the same metrics in a similar fashion.

Also, CoordinatorLoaderImpl#load will now return LoadSummary which encapsulates the start time, end time, number of records/bytes.

Co-authored-by: David Jacot <djacot@confluent.io>

Reviewers:  Ritika Reddy <rreddy@confluent.io>, Calvin Liu <caliu@confluent.io>, David Jacot <djacot@confluent.io>, Justine Olshan <jolshan@confluent.io>
2023-10-17 16:06:23 -07:00
Omnia G.H Ibrahim 9af1e74b5e
KAFKA-14596: Move TopicCommand to tools (#13201)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Federico Valeri <fedevaleri@gmail.com>
2023-10-17 11:40:15 +02:00
Ismael Juma 69e591db3a
MINOR: Rewrite/Move KafkaNetworkChannel to the `raft` module (#14559)
This is now possible since `InterBrokerSend` was moved from `core` to `server-common`.
Also rewrite/move `KafkaNetworkChannelTest`.

The scala version of `KafkaNetworkChannelTest` passed with the changes here (before I
deleted it).

Reviewers: Justine Olshan <jolshan@confluent.io>, José Armando García Sancio <jsancio@users.noreply.github.com>
2023-10-16 20:10:31 -07:00
Lianet Magrans 58dfa1cc81
MINOR - KAFKA-15550: Validation for negative target times in offsetsForTimes (#14503)
The current KafkaConsumer offsetsForTimes fails with IllegalArgumentException if negative target timestamps are provided as arguments. This change includes the same validation and tests for the new consumer implementation (and some improved comments for the updateFetchPositions)

Reviewer: Lucas Brutschy <lbrutschy@confluent.io>
2023-10-13 09:59:57 +02:00
Jeff Kim 7b5d640cc6
KAFKA-14987; Implement Group/Offset expiration in the new coordinator (#14467)
This patch implements the groups and offsets expiration in the new group coordinator.

Reviewers: Ritika Reddy <rreddy@confluent.io>, David Jacot <djacot@confluent.io>
2023-10-11 23:45:13 -07:00
Mayank Shekhar Narula d817b1b590
KAFKA-15415: On producer-batch retry, skip-backoff on a new leader (#14384)
When producer-batch is being retried, new-leader is known for the partition Vs the leader used in last attempt, then it is worthwhile to retry immediately to this new leader. A partition-leader is considered to be newer, if the epoch has advanced.

Reviewers: Walker Carlson <wcarlson@apache.org>, Kirk True <kirk@kirktrue.pro>, Andrew Schofield <andrew_schofield@uk.ibm.com
2023-10-05 09:11:47 -05:00
Dongnuo Lyu a12f9f97c9
KAFKA-14506: Implement DeleteGroups API and OffsetDelete API (#14408)
This patch implements DeleteGroups and OffsetDelete API in the new group coordinator.

Reviewers: yangy0000, Ritika Reddy <rreddy@confluent.io>, Jeff Kim <jeff.kim@confluent.io>, David Jacot <djacot@confluent.io>
2023-10-04 02:30:45 -07:00
Nikolay 8f8dbad564
KAFKA-14595 ReassignPartitionsIntegrationTest rewritten in java (#14456)
This PR is part of #13247
It contains ReassignPartitionsIntegrationTest rewritten in java.
Goal of PR is reduce changes size in main PR.

Reviewers: Taras Ledkov  <tledkov@apache.org>, Justine Olshan <jolshan@confluent.io>
2023-10-02 13:22:17 -07:00
Lucas Brutschy 6263197a62
KAFKA-15326: [9/N] Start and stop executors and cornercases (#14281)
* Implements start and stop of task executors
* Introduce flush operation to keep consumer operations out of the processing threads
* Fixes corner case: handle requested unassignment during shutdown
* Fixes corner case: handle race between voluntary unassignment and requested unassigment
* Fixes corner case: task locking future completes for the empty set
* Fixes corner case: we should not reassign a task with an uncaught exception to a task executor
* Improved logging
* Number of threads controlled from outside, of the TaskManager

Reviewers: Bruno Cadonna <bruno@confluent.io>
2023-10-02 15:41:21 +02:00
Colin Patrick McCabe fcac880fd5
KAFKA-15466: Add KIP-919 support for some admin APIs (#14399)
Add support for --bootstrap-controller in the following command-line tools:
    - kafka-cluster.sh
    - kafka-configs.sh
    - kafka-features.sh
    - kafka-metadata-quorum.sh

To implement this, the following AdminClient APIs now support the new bootstrap.controllers
configuration:
    - Admin.alterConfigs
    - Admin.describeCluster
    - Admin.describeConfigs
    - Admin.describeFeatures
    - Admin.describeMetadataQuorum
    - Admin.incrementalAlterConfigs
    - Admin.updateFeatures

Command-line tool changes:
    - Add CommandLineUtils.initializeBootstrapProperties to handle parsing --bootstrap-controller
      in addition to --bootstrap-server.
    - Add --bootstrap-controller to ConfigCommand.scala, ClusterTool.java, FeatureCommand.java, and
      MetadataQuorumCommand.java.

KafkaAdminClient changes:
    - Add the AdminBootstrapAddresses class to handle extracting bootstrap.servers or
      bootstrap.controllers from the config map for KafkaAdminClient.
    - In AdminMetadataManager, store the new usingBootstrapControllers boolean. Generalize
      authException to encompass the concept of fatal exceptions in general. (For example, the
      fatal exception where we talked to the wrong node type.) Treat
      MismatchedEndpointTypeException and UnsupportedEndpointTypeException as fatal exceptions.
    - Extend NodeProvider to include information about whether bootstrap.controllers is supported.
    - Modify the APIs described above to support bootstrap.controllers.

Server-side changes:
    - Support DescribeConfigsRequest on kcontrollers.
    - Add KRaftMetadataCache to the kcontroller to simplify implemeting describeConfigs (and
      probably more APIs in the future). It's mainly a wrapper around MetadataImage, so there is
      essentially no extra resource consumption.
    - Split RuntimeLoggerManager out of ConfigAdminManager to handle the incrementalAlterConfigs
      support for BROKER_LOGGER. This is now supported on kcontrollers as well as brokers.
    - Fix bug in AuthHelper.computeDescribeClusterResponse that resulted in us always sending back
      BROKER as the endpoint type, even on the kcontroller.

Miscellaneous:
    - Fix a few places in exceptions and log messages where we wrote "broker" instead of "node".
      For example, an exception in NodeApiVersions.java, and a log message in NetworkClient.java.
    - Fix the slf4j log prefix used by KafkaRequestHandler logging so that request handlers on a
      controller don't look like they're on a broker.
    - Make the FinalizedVersionRange constructor public for the sake of a junit test.
    - Add unit and integration tests for the above.

Reviewers: David Arthur <mumrah@gmail.com>, Doguscan Namal <namal.doguscan@gmail.com>
2023-09-26 14:43:42 -07:00
Ismael Juma 7ba6d7a0b4
MINOR: Update to Scala 2.13.12 (#14430)
It offers a quickfix action for certain errors, includes a number of bug fixes and it
introduces a new warning by default (https://github.com/scala/scala/pull/10462).

In addition to the scala version bump, we also fix the new compiler warnings and
bump the scalafmt version (the previous version failed with the new scala version).

Release notes: https://github.com/scala/scala/releases/tag/v2.13.12

Reviewers: Divij Vaidya <diviv@amazon.com>, Satish Duggana <satishd@apache.org>
2023-09-24 06:05:12 -07:00
Nikolay daf8a0deda
KAFKA-14595 ReassignPartitionsUnitTest rewritten in java (#14355)
This PR is part of #13247
It contains changes to rewrite single test in java.
Intention is reduce changes in parent PR.

Reviewers: Luke Chen <showuon@gmail.com>, Taras Ledkov <tledkov@apache.org>
2023-09-23 09:45:14 +08:00
Tyler Bertrand eea1854479
KAFKA-15476: Resolves cache misses in checkstyle (#14344)
Resolves cache misses in checkstyle tasks due to absolute paths in configProperties.

Sets configDirectory extension property, which is made available by the checkstyle plugin as ${config_loc} in the checkstyle xml files, as shown in the Checkstyle Gradle docs. The absolute paths set in configProperties are then replaced by relative paths from configDirectory. Because the header and suppression config file names are static and only referenced once, these were removed from configProperties and the file names are given directly in checkstyle.xml

Reviewers: Divij Vaidya <diviv@amazon.com>
2023-09-19 10:51:57 +02:00
Kirk True e1dc6d9f34
KAFKA-14274 [2-5/7]: Introduction of more infrastructure for forthcoming fetch request manager (#14359)
This continues the work of providing the groundwork for the fetch
refactoring work by introducing some new classes and refactoring the
existing code to use the new classes where applicable.

Changes:

* Minor clean up of the events classes to make data immutable,
  private, and implement toString().
* Added IdempotentCloser which prevents a resource from being closed
  more than once. It's general enough that it could be used elsewhere
  in the project, but it's limited to the consumer internals for now.
* Split core Fetcher code into classes to buffer raw results
  (FetchBuffer) and to collect raw results into ConsumerRecords
  (FetchCollector). These can be tested and changed in isolation from
  the core fetcher logic.
* Added NodeStatusDetector which abstracts methods from
  ConsumerNetworkClient so that it and NetworkClientDelegate can be
  used in AbstractFetch via the interface instead of using
  ConsumerNetworkClient directly.

Reviewers: Jun Rao <junrao@gmail.com>
2023-09-16 09:15:37 -07:00
zhaohaidao f309299f3c
KAFKA-14503: Implement ListGroups (#14271)
This patch implements the ListGroups API in the new group coordinator.

Reviewers: David Jacot <djacot@confluent.io>
2023-09-14 23:45:03 -07:00
Jeff Kim e9057aab37
KAFKA-14502; Implement LeaveGroup protocol in new GroupCoordinator (#14147)
This patch implements the LeaveGroup API in the new group coordinator.

Reviewers: David Jacot <djacot@confluent.io>
2023-09-13 01:43:37 -07:00
Nikolay 0029bc4897
KAFKA-14595: ReassignPartitionsCommandArgsTest rewritten in java (#14217)
Reviewers: Taras Ledkov <tledkov@apache.org>, Greg Harris <greg.harris@aiven.io>
2023-09-07 10:12:07 -07:00
Andrew Schofield b49013b73e
KAFKA-9800: Exponential backoff for Kafka clients - KIP-580 (#14111)
Implementation of KIP-580 to add exponential back-off to situations in which retry.backoff.ms
is used to delay backoff attempts. This KIP adds exponential backoff behavior with a maximum
controlled by a new config retry.backoff.max.ms, together with a +/- 20% of jitter to spread the
retry attempts of the client fleet.

Reviewers: Mayank Shekhar Narula <mayanks.narula@gmail.com>, Milind Luthra <i.milind.luthra@gmail.com>, Kirk True <kirk@mustardgrain.com>, Jun Rao<junrao@gmail.com>
2023-09-05 11:57:51 -07:00
Kamal Chandraprakash 4590d565ef
KAFKA-15399: Enable OffloadAndConsumeFromLeader test (#14285)
Reviewers: Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>, Satish Duggana <satishd@apache.org>
2023-08-28 12:29:50 +02:00
Satish Duggana d4ab3ae85a
KAFKA-14888: Added remote log segments retention mechanism based on time and size. (#13561)
This change introduces a remote log segment segment retention cleanup mechanism.

RemoteLogManager runs retention cleanup activity tasks on each leader replica. It assesses factors such as overall size and retention duration, subsequently removing qualified segments from remote storage. This process also involves adjusting the log-start-offset within the UnifiedLog accordingly. It also cleans up the segments which have epochs earlier than the earliest leader epoch in the current leader. 

Co-authored-by: Satish Duggana <satishd@apache.org>
Co-authored-by: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>

Reviewers: Jun Rao <junrao@gmail.com>, Divij Vaidya <diviv@amazon.com, Luke Chen <showuon@gmail.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Christo Lolov <lolovc@amazon.com>, Jorge Esteban Quilcate Otoya <quilcate.jorge@gmail.com>, Alexandre Dupriez <alexandre.dupriez@gmail.com>, Nikhil Ramakrishnan <ramakrishnan.nikhil@gmail.com>
2023-08-25 05:27:59 +05:30
Kamal Chandraprakash 6492164d9c
KAFKA-15167: Tiered Storage Test Harness Framework (#14116)
`TieredStorageTestHarness` is a base class for integration tests exercising the tiered storage functionality. This uses  `LocalTieredStorage` instance as the second-tier storage system and `TopicBasedRemoteLogMetadataManager` as the remote log metadata manager.

Co-authored-by: Alexandre Dupriez <alexandre.dupriez@gmail.com>
Co-authored-by: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
2023-08-20 17:15:52 +05:30
Proven Provenzano c2759df067
KAFKA-15219: KRaft support for DelegationTokens (#14083)
Reviewers: David Arthur <mumrah@gmail.com>, Ron Dagostino <rndgstn@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>, Viktor Somogyi <viktor.somogyi@cloudera.com>
2023-08-19 14:01:08 -04:00
Greg Harris f5655d31d3
KAFKA-15030: Add connect-plugin-path command-line tool (#14064)
Reviewers: Chris Egerton <chrise@aiven.io>
2023-08-11 12:05:51 -07:00
Greg Harris ab60bce090
KAFKA-15239: Fix ThroughputThrottler import-control (#14188)
Reviewers: Chris Egerton <chrise@aiven.io>
2023-08-10 16:53:49 -07:00
Nikolay ddeb89f4a9
KAFKA-14595: Move AdminUtils to server-common (#14096)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2023-08-09 10:32:45 +02:00
Luke Chen 748175ce62
KAFKA-15189: only init remote topic metrics when enabled (#14133)
Only initialize remote topic metrics when system-wise remote storage is enabled to avoid impacting performance for existing brokers. Also add tests.

Reviewers: Divij Vaidya <diviv@amazon.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
2023-08-05 13:00:16 +08:00
Ivan Yurchenko b3db905b27
KAFKA-15107: Support custom metadata for remote log segment (#13984)
* KAFKA-15107: Support custom metadata for remote log segment

This commit does the changes discussed in the KIP-917. Mainly, changes the `RemoteStorageManager` interface in order to return `CustomMetadata` and then ensures these custom metadata are stored, propagated, (de-)serialized correctly along with the standard metadata throughout the whole lifecycle. It introduces the `remote.log.metadata.custom.metadata.max.size` to limit the custom metadata size acceptable by the broker and stop uploading in case a piece of metadata exceeds this limit.

On testing:
1. `RemoteLogManagerTest` checks the case when a piece of custom metadata is larger than the configured limit.
2. `RemoteLogSegmentMetadataTest` checks if `createWithUpdates` works correctly, including custom metadata.
3. `RemoteLogSegmentMetadataTransformTest`, `RemoteLogSegmentMetadataSnapshotTransformTest`, and `RemoteLogSegmentMetadataUpdateTransformTest` were added to test the corresponding class (de-)serialization, including custom metadata.
4. `FileBasedRemoteLogMetadataCacheTest` checks if custom metadata are being correctly saved and loaded to a file (indirectly, via `equals`).
5. `RemoteLogManagerConfigTest` checks if the configuration setting is handled correctly.

Reviewers: Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>, Divij Vaidya <diviv@amazon.com>
2023-08-04 18:23:25 +05:30
Jeff Kim 19f9e1e6d0
KAFKA-14501: Implement Heartbeat protocol in new GroupCoordinator (#14056)
This patch implements the existing Heartbeat API in the new Group Coordinator.

Reviewers: David Jacot <djacot@confluent.io>
2023-07-28 15:13:27 +02:00
Hao Li ed44bcd71b
KAFKA-15022: [3/N] use graph to compute rack aware assignment for active stateful tasks (#14030)
Part of KIP-925.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2023-07-26 16:02:52 -07:00
Federico Valeri bb677c4959
KAFKA-14583: Move ReplicaVerificationTool to tools (#14059)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2023-07-26 12:04:34 +02:00
Colin Patrick McCabe c7de30f38b
KAFKA-15183: Add more controller, loader, snapshot emitter metrics (#14010)
Implement some of the metrics from KIP-938: Add more metrics for
measuring KRaft performance.

Add these metrics to QuorumControllerMetrics:
    kafka.controller:type=KafkaController,name=TimedOutBrokerHeartbeatCount
    kafka.controller:type=KafkaController,name=EventQueueOperationsStartedCount
    kafka.controller:type=KafkaController,name=EventQueueOperationsTimedOutCount
    kafka.controller:type=KafkaController,name=NewActiveControllersCount

Create LoaderMetrics with these new metrics:
    kafka.server:type=MetadataLoader,name=CurrentMetadataVersion
    kafka.server:type=MetadataLoader,name=HandleLoadSnapshotCount

Create SnapshotEmitterMetrics with these new metrics:
    kafka.server:type=SnapshotEmitter,name=LatestSnapshotGeneratedBytes
    kafka.server:type=SnapshotEmitter,name=LatestSnapshotGeneratedAgeMs

Reviewers: Ron Dagostino <rndgstn@gmail.com>
2023-07-24 21:13:58 -07:00
David Jacot 2528dd4116
KAFKA-14499: [2/N] Add OffsetCommit record & related (#14047)
This patch does a few things:
1) It introduces the `OffsetAndMetadata` class which hold the committed offsets in the group coordinator.
2) It adds methods to deal with OffsetCommit records to `RecordHelpers`.
3) It adds `MetadataVersion#offsetCommitValueVersion` to get the version of the OffsetCommit value record that should be used.

Reviewers: Jeff Kim <jeff.kim@confluent.io>, David Arthur <mumrah@gmail.com>, Justine Olshan <jolshan@confluent.io>
2023-07-21 20:09:06 +02:00
Yash Mayya 4daeb2714c
KAFKA-13431 (KIP-793): Expose the original pre-transform topic partition and offset in sink records (#14024)
Reviewers: Greg Harris <greg.harris@aiven.io>, Chris Egerton <chrise@aiven.io>
2023-07-21 12:06:01 -04:00
Luke Chen 27ea025e33
KAFKA-15176: add tests for tiered storage metrics (#13999)
Added tests for metrics:
1. RemoteLogReaderTaskQueueSize
2. RemoteLogReaderAvgIdlePercent
3. RemoteLogManagerTasksAvgIdlePercent

Also, added tests for OffsetOutOfRangeException will be thrown while reading logs

Reviewers: Christo Lolov <christololov@gmail.com>, Satish Duggana <satishd@apache.org>
2023-07-21 10:30:33 +08:00
Greg Harris 125dbb9286
KAFKA-14760: Move ThroughputThrottler from tools to clients, remove tools dependency from connect-runtime (#13313)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2023-07-20 12:58:48 -07:00
Federico Valeri 334c41d604
KAFKA-14734: Use CommandDefaultOptions in StreamsResetter (#13983)
This PR adds CommandDefaultOptions usage like in the other joptsimple based tools. It also moves the associated unit test class from streams to tools module as discussed in #13127 (comment)

Reviewers:  Luke Chen <showuon@gmail.com>, Bruno Cadonna <cadonna@apache.org>, Sagar Rao <sagarmeansocean@gmail.com>
2023-07-20 18:45:05 +08:00
Jeff Kim a500c3ecf9
KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator (#13870)
This patch implements the existing JoinGroup protocol within the new group coordinator. 

Some notable differences:
* Methods return a CoordinatorResult to the runtime framework, which includes records to append to the log as well as a future to complete after the append succeeds/fails.
* The coordinator runtime ensures that only a single thread will be processing a group at any given time, therefore there is no more locking on groups.
* Instead of using on purgatories, we rely on the Timer interface to schedule/cancel delayed operations.

Reviewers: David Jacot <djacot@confluent.io>
2023-07-19 09:15:13 +02:00
Abhijeet Kumar fd3b1137d2
KAFKA-14953: Add tiered storage related metrics (#13944)
* KAFKA-14953: Adding RemoteLogManager metrics
In this PR, I have added the following metrics that are related to tiered storage mentioned in[ KIP-405](https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage).
|Metric|Description|
|-----------------------------------------|--------------------------------------------------------------|
| RemoteReadRequestsPerSec                    | Number of remote storage read requests per second               |
| RemoteWriteRequestsPerSec                    | Number of remote storage write requests per second              |
| RemoteBytesInPerSec                                | Number of bytes read from remote storage per second           |
| RemoteReadErrorsPerSec                          | Number of remote storage read errors per second                   |
| RemoteBytesOutPerSec                             | Number of bytes copied to remote storage per second            |
| RemoteWriteErrorsPerSec                          | Number of remote storage write errors per second                   |
| RemoteLogReaderTaskQueueSize             | Number of remote storage read tasks pending for execution.  |
| RemoteLogReaderAvgIdlePercent             | Average idle percent of the remote storage reader thread pool|
| RemoteLogManagerTasksAvgIdlePercent | Average idle percent of RemoteLogManager thread pool          |

Added unit tests for all the rate metrics.

Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Jorge Esteban Quilcate Otoya <quilcate.jorge@gmail.com>, Staniel Yao<yaolixinylx@gmail.com>, hudeqi<1217150961@qq.com>, Satish Duggana <satishd@apache.org>
2023-07-18 20:16:19 +05:30
Omnia G H Ibrahim 0c6b1a4e9a
KAFKA-14737: Move kafka.utils.json to server-common (#13585)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Federico Valeri <fedevaleri@gmail.com>
2023-07-18 11:02:40 +02:00
vamossagar12 fa5b493241
KAFKA-14647: Move TopicFilter to server-common/utils (#13158)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Federico Valeri <fedevaleri@gmail.com>
2023-07-18 10:38:56 +02:00
David Jacot 32ff347b2c
KAFKA-14462; [23/23] Wire GroupCoordinatorService in BrokerServer (#13991)
This patch wires the new group coordinator in BrokerServer (KRaft only). With this, it is now possible to run a cluster with the new group coordinator and to use the ConsumerGroupHeartbeat API by specifying the following two properties:
- group.coordinator.new.enable = true (to enable the new group coordinator)
- unstable.api.versions.enable = true (to enable unreleased APIs)

Note that the new group coordinator does not support all the existing APIs yet.

Reviewers: Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>
2023-07-14 17:41:06 +02:00
Satish Duggana 7e2f878713
KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module. (#13275)
KAFKA-14522 Rewrite and Move of RemoteIndexCache to storage module.
Cleanedup index file suffix usages and other minor cleanups

Reviewers: Jun Rao <junrao@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Jorge Esteban Quilcate Otoya <quilcate.jorge@gmail.com>
2023-07-11 23:55:23 +05:30
David Jacot bd1f02b2be
MINOR: Move MockTimer to server-common (#13954)
This patch rewrites MockTimer in Java and moves it from core to server-common. This continues the work started in https://github.com/apache/kafka/pull/13820.

Reviewers: Divij Vaidya <diviv@amazon.com>
2023-07-06 14:56:05 +02:00
David Jacot 98fbd8afc7
KAFKA-14462; [20/N] Refresh subscription metadata on new metadata image (#13901)
This patch adds (1) the logic to propagate a new MetadataImage to the running coordinators; and (2) the logic to ensure that all the consumer groups subscribed to topics with changes will refresh their subscriptions metadata on the next heartbeat. In the mean time, it ensures that freshly loaded consumer groups also refresh their subscriptions metadata on the next heartbeat.

Reviewers: Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>
2023-07-05 18:28:38 +02:00
Kirk True a81f35c1c8
KAFKA-14831: Illegal state errors should be fatal in transactional producer (#13591)
Poison the transaction manager if we detect an illegal transition in the Sender thread. A ThreadLocal in is stored in TransactionManager so that the Sender can inform TransactionManager which thread it's using.

Reviewers: Daniel Urban <durban@cloudera.com>, Justine Olshan <jolshan@confluent.io>, Jason Gustafson <jason@confluent.io>
2023-06-29 11:21:15 -07:00
Yash Mayya 6e72986949
KAFKA-14784: Connect offset reset REST API (#13818)
Reviewers: Chris Egerton <chrise@aiven.io>
2023-06-23 13:27:46 -04:00
David Jacot a81486e4f8
KAFKA-14462; [18/N] Add GroupCoordinatorService (#13812)
This patch introduces the GroupCoordinatorService. This is the new (incomplete) implementation of the group coordinator based on the coordinator runtime introduced in https://github.com/apache/kafka/pull/13795.

Reviewers: Divij Vaidya <diviv@amazon.com>, Justine Olshan <jolshan@confluent.io>
2023-06-22 09:06:10 +02:00
Dimitar Dimitrov b100f1efac
KAFKA-15087 Move/rewrite InterBrokerSendThread to server-commons (#13856)
The Java rewrite is kept relatively close to the Scala original
to minimize potential newly introduced bugs and to make reviewing
simpler. The following details might be of note:
- The `Logging` trait moved to InterBrokerSendThread with the
rewrite of ShutdownableThread has been similarly moved to any
subclasses that currently use it. InterBrokerSendThread's own
logging has been made to use ShutdownableThread's logger which
mimics the prefix/log identifier that the trait provided.
- The case RequestAndCompletionHandler class has been made a
separate POJO class and the internal-use UnsentRequests class
has been kept as a static nested class.
- The relatively commonly used but internal (not part of the
public API) clients classes that InterBrokerSendThread relies on
have been allowlisted in the server-common import control.
- The accompanying test class has also been moved and rewritten
with one new test added and most of the pre-existing tests made
stricter.

Reviewers: David Jacot <djacot@confluent.io>
2023-06-20 16:50:46 +02:00
Colin P. McCabe cd3c0ab1a3 KAFKA-15060: fix the ApiVersionManager interface
This PR expands the scope of ApiVersionManager a bit to include returning the current
MetadataVersion and features that are in effect. This is useful in general because that information
needs to be returned in an ApiVersionsResponse. It also allows us to fix the ApiVersionManager
interface so that all subclasses implement all methods of the interface. Having subclasses that
don't implement some methods is dangerous because they could cause exceptions at runtime in
unexpected scenarios.

On the KRaft controller, we were previously performing a read operation in the QuorumController
thread to get the current metadata version and features. With this PR, we now read a volatile
variable maintained by a separate MetadataVersionContextPublisher object. This will improve
performance and simplify the code. It should not change the guarantees we are providing; in both
the old and new scenarios, we need to be robust against version skew scenarios during updates.

Add a Features class which just has a 3-tuple of metadata version, features, and feature epoch.
Remove MetadataCache.FinalizedFeaturesAndEpoch, since it just duplicates the Features class.
(There are some additional feature-related classes that can be consolidated in in a follow-on PR.)

Create a java class, EndpointReadyFutures, for managing the futures associated with individual
authorizer endpoints. This avoids code duplication between ControllerServer and BrokerServer and
makes this code unit-testable.

Reviewers: David Arthur <mumrah@gmail.com>, dengziming <dengziming1993@gmail.com>, Luke Chen <showuon@gmail.com>
2023-06-19 16:46:44 -07:00
David Jacot 7eea2a3908
MINOR: Move MockTime to server-common (#13823)
This patch rewrite `MockTime` in Java and moves it to `server-common` module. This is a prerequisite to move `MockTimer` later on to `server-common` as well. 

Reviewers: David Arthur <mumrah@gmail.com>
2023-06-09 08:54:25 +02:00
Lianet Magrans 4af4bccbbf
KAFKA-14966: Extract OffsetFetcher reusable logic (#13815)
The OffsetFetcher is internally used by the KafkaConsumer to fetch offsets, validate and reset positions. For the new KafkaConsumer with a refactored threading model, similar functionality will be needed.

This is an initial refactoring for extracting logic from the OffsetFetcher, that will be reused by the new consumer implementation. No changes to the existing logic, just extracting classes, functions or pieces of logic.

All the functionality moved out of the OffsetFetcher is already covered by tests in OffsetFetcherTest and FetcherTest. There were no individual tests for the extracted functions, so no tests were migrated.

Reviewers: Jun Rao <junrao@gmail.com>
2023-06-08 14:03:45 -07:00
David Jacot 47551ea369
KAFKA-14462; [13/N] CoordinatorEvent and CoordinatorEventProcessor (#13666)
Adds CoordinatorEvent, CoordinatorEventProcessor, and MultiThreadedEventProcessor.

Reviewers: Kirk True <ktrue@confluent.io>, Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>
2023-06-01 13:33:40 -07:00
Yash Mayya 9bb2f78d53
KAFKA-15034: Improve performance of the ReplaceField SMT; add JMH benchmark (#13776)
Reviewers: Chris Egerton <chrise@aiven.io>
2023-06-01 15:14:31 -04:00
David Jacot 49d9c6775d
KAFKA-14462; [12/N] Add GroupMetadataManager and ConsumerGroup (#13639)
This patch adds the GroupMetadataManager to the group-coordinator module. This manager is responsible for handling the groups management, the members management and the entire reconciliation process. At this point, only the new consumer group type/protocol is implemented.

The new manager is based on an architecture inspired from the quorum controller. A request can access/read the state but can't mutate it directly. Instead, a list of records is generated together with the response and those records are applied to the state by the runtime framework. We use timeline data structures. Note that the runtime framework is not part of this patch. It will come in a following one.

Reviewers: Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>
2023-05-31 08:29:41 +02:00
Colin Patrick McCabe b74204fa0a
KAFKA-14996: Handle overly large user operations on the kcontroller (#13742)
Previously, if a user tried to perform an overly large batch operation on the KRaft controller
(such as creating a million topics), we would create a very large number of records in memory. Our
attempt to write these records to the Raft layer would fail, because there were too many to fit in
an atomic batch. This failure, in turn, would trigger a controller failover.

(Note: I am assuming here that no topic creation policy was in place that would prevent the
creation of a million topics. I am also assuming that the user operation must be done atomically,
which is true for all current user operations, since we have not implemented KIP-868 yet.)

With this PR, we fail immediately when the number of records we have generated exceeds the
threshold that we can apply. This failure does not generate a controller failover. We also now
fail with a PolicyViolationException rather than an UnknownServerException.

In order to implement this in a simple way, this PR adds the BoundedList class, which wraps any
list and adds a maximum length. Attempts to grow the list beyond this length cause an exception to
be thrown.

Reviewers: David Arthur <mumrah@gmail.com>, Ismael Juma <ijuma@apache.org>, Divij Vaidya <diviv@amazon.com>
2023-05-26 13:16:17 -07:00
Yash Mayya 7ff2dbb107
KAFKA-14368: Connect offset write REST API (#13465)
Reviewers: Greg Harris <greg.harris@aiven.io>, Chris Egerton <chrise@aiven.io>
2023-05-26 12:08:06 -04:00
Colin P. McCabe 12130cfcec MINOR: Create the MetadataNode classes to introspect MetadataImage
Metadata image classes such as MetadataImage, ClusterImage, FeaturesImage, and so forth contain
numerous sub-images. This PR adds a structured way of traversing those sub-images. This is useful
for the metadata shell, and also for implementing toString functions.

In both cases, the previous solution was suboptimal. The metadata shell was previously implemented
in an ad-hoc way by mutating text-based tree nodes when records were replayed. This was difficult
to keep in sync with changes to the record types (for example, we forgot to do this for SCRAM). It
was also pretty low-level, being done at a level below that of the image classes. For toString, it
was difficult to keep the implementations consistent previously, and also support both redacted and
non-redacted output.

The metadata shell directory was getting crowded since we never had submodules for it. This PR
creates glob/, command/, node/, and state/ directories to keep things better organized.

Reviewers: David Arthur <mumrah@gmail.com>, Ron Dagostino <rdagostino@confluent.io>
2023-05-23 10:11:26 -07:00
Jeff Kim c98c1ed41c
KAFKA-14500; [3/N] add GroupMetadataKey/Value record helpers (#13704)
This path enables the new group metadata manager to generate GroupMetadataKey/Value records.

Reviewers: David Jacot <djacot@confluent.io>
2023-05-23 10:42:13 +02:00
Satish Duggana 6f19730164
KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory. (#13535)
This change includes
- Recognize the fetch requests with out of range local log offsets
- Add fetch implementation for the data lying in the range of [logStartOffset, localLogStartOffset]
- Add a new purgatory for async remote read requests which are served through a specific thread pool

We have an extended version of remote fetch that can fetch from multiple remote partitions in parallel, which we will raise as a followup PR.

A few tests for the newly introduced changes are added in this PR. There are some tests available for these scenarios in 2.8.x, refactoring with the trunk changes, will add them in followup PRs.

Other contributors:
Kamal Chandraprakash <kchandraprakash@uber.com> - Further improvements and adding a few tests
Luke Chen <showuon@gmail.com> - Added a few test cases for these changes.

PS: This functionality is pulled out from internal branches with other functionalities related to the feature in 2.8.x. The reason for not pulling all the changes as it makes the PR huge, and burdensome to review and it also needs other metrics, minor enhancements(including perf), and minor changes done for tests. So, we will try to have followup PRs to cover all those.

Reviewers: Jun Rao <junrao@gmail.com>, Alexandre Dupriez <alexandre.dupriez@gmail.com>, Divij Vaidya <diviv@amazon.com>,  Jorge Esteban Quilcate Otoya <quilcate.jorge@gmail.com>
2023-05-18 06:37:37 +05:30
Jeff Kim cc011f77aa
KAFKA-14500; [2/N] Rewrite GroupMetadata in Java (#13663)
This patch introduces `GenericGroup` which rewrite the `GroupMetadata` in Java. The `GenericGroup` is basically a group using the current rebalance protocol in the new group coordinator.

Reviewers: Ritika Reddy <rreddy@confluent.io>, Christo Lolov <lolovc@amazon.com>, David Jacot <djacot@confluent.io>
2023-05-12 11:22:29 +02:00
Federico Valeri c757af5f7c
KAFKA-14752: Kafka examples improvements - demo changes (#13517)
KAFKA-14752: Kafka examples improvements - demo changes

Reviewers: Luke Chen <showuon@gmail.com>
2023-05-12 10:39:12 +08:00
David Arthur 0822ce0ed1
KAFKA-14840: Support for snapshots during ZK migration (#13461)
This patch adds support for handling metadata snapshots while in dual-write mode. Prior to this change, if the active
controller loaded a snapshot, it would get out of sync with the ZK state.

In order to reconcile the snapshot state with ZK, several methods were added to scan through the metadata in ZK to
compute differences with the MetadataImage. Since this introduced a lot of code, I opted to split out a lot of methods
from ZkMigrationClient into their own client interfaces, such as TopicMigrationClient, ConfigMigrationClient, and
AclMigrationClient. Each of these has some iterator method that lets the caller examine the ZK state in a single pass
and without using too much memory.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, Luke Chen <showuon@gmail.com>
2023-05-05 01:35:26 -07:00
Proven Provenzano e29942347a
KAFKA-14859: SCRAM ZK to KRaft migration with dual write (#13628)
Handle migrating SCRAM records in ZK when migrating from ZK to KRaft.

This includes handling writing back SCRAM records to ZK while in dual write mode where metadata updates are written to both the KRaft metadata log and to ZK. This allows for rollback of migration to include SCRAM metadata changes.

Reviewers: David Arthur <mumrah@gmail.com>
2023-05-01 09:56:04 -04:00
Luke Chen d796480fe8
KAFKA-14909: check zkMigrationReady tag before migration (#13631)
1. add ZkMigrationReady in apiVersionsResponse
2. check all nodes if ZkMigrationReady are ready before moving to next migration state

Reviewers: David Arthur <mumrah@gmail.com>, dengziming <dengziming1993@gmail.com>
2023-04-28 14:35:12 +08:00
David Arthur c1b5c75d92
KAFKA-14805 KRaft controller supports pre-migration mode (#13407)
This patch adds the concept of pre-migration mode to the KRaft controller. While in this mode, 
the controller will only allow certain write operations. The purpose of this is to disallow metadata 
changes when the controller is waiting for the ZK migration records to be committed.

The following ControllerWriteEvent operations are permitted in pre-migration mode

* completeActivation
* maybeFenceReplicas
* writeNoOpRecord
* processBrokerHeartbeat
* registerBroker (only for migrating ZK brokers)
* unregisterBroker

Raft events and other controller events do not follow the same code path as ControllerWriteEvent, 
so they are not affected by this new behavior.

This patch also add a new metric as defined in KIP-868: kafka.controller:type=KafkaController,name=ZkMigrationState

In order to support upgrades from 3.4.0, this patch also redefines the enum value of value 1 to mean 
MIGRATION rather than PRE_MIGRATION.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
2023-04-26 10:20:30 -04:00
David Jacot 9a36da12b7
KAFKA-14462; [8/N] Add ConsumerGroupMember (#13538)
This patch adds ConsumerGroupMember.

Reviewers: Christo Lolov <lolovc@amazon.com>, Jeff Kim <jeff.kim@confluent.io>, Jason Gustafson <jason@confluent.io>
2023-04-25 18:50:51 +02:00
Gantigmaa Selenge ea540fa400
KAFKA-14592: Move FeatureCommand to tools (#13459)
KAFKA-14592: Move FeatureCommand to tools

Reviewers: Luke Chen <showuon@gmail.com>
2023-04-25 20:28:37 +08:00
David Jacot 2d0b816150
MINOR: Move `ControllerPurgatory` to `server-common` (#13555)
This patch renames from `ControllerPurgatory` to `DeferredEventQueue` and moves it from the `metadata` module to `server-common` module.

Reviewers: Alexandre Dupriez <alexandre.dupriez@gmail.com>, Ziming Deng <dengziming1993@gmail.com>, José Armando García Sancio <jsancio@apache.org>
2023-04-21 11:19:04 +02:00
Proven Provenzano abca86511e
KAFKA-14881: Rework UserScramCredentialRecord (#13513)
Rework UserScramCredentialRecord to store serverKey and StoredKey rather than saltedPassword. This
is necessary to support migration from ZK, since those are the fields we stored in ZK.  Update
latest MetadataVersion to IBP_3_5_IV2 and make SCRAM support conditional on this version.  Moved
ScramCredentialData.java from org.apache.kafka.image to org.apache.kafka.metadata, which seems more
appropriate.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2023-04-18 09:41:38 -07:00
Ron Dagostino e27926f92b
KAFKA-14735: Improve KRaft metadata image change performance at high … (#13280)
topic counts.

Introduces the use of persistent data structures in the KRaft metadata image to avoid copying the entire TopicsImage upon every change.  Performance that was O(<number of topics in the cluster>) is now O(<number of topics changing>), which has dramatic time and GC improvements for the most common topic-related metadata events.  We abstract away the chosen underlying persistent collection library via ImmutableMap<> and ImmutableSet<> interfaces and static factory methods.

Reviewers: Luke Chen <showuon@gmail.com>, Colin P. McCabe <cmccabe@apache.org>, Ismael Juma <ismael@juma.me.uk>, Purshotam Chauhan <pchauhan@confluent.io>
2023-04-17 17:52:28 -04:00
Victoria Xia 1d5d003ff4
KAFKA-14834: [5/N] Drop out-of-order records from FK join with versioned tables (#13522)
This PR updates foreign-key table-table join processors to ignore out-of-order records from versioned tables, as specified in KIP-914.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2023-04-12 19:05:10 -07:00
Victoria Xia 1395ad6497
KAFKA-14834: [4/N] Drop out-of-order records from table-table join with versioned tables (#13510)
This PR updates primary-key table-table join processors to ignore out-of-order records from versioned tables, as specified in KIP-914.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2023-04-12 17:06:28 -07:00
David Jacot e1e3900ba1
KAFKA-14462; [4/N] Add Group, Record and Result (#13520)
This patch adds Group, Record and Result.

Reviewers: Jason Gustafson <jason@confluent.io>, Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>
2023-04-12 13:16:49 +02:00
Satish Duggana e99984248d
KAFKA-9550 Copying log segments to tiered storage in RemoteLogManager (#13487)
Added functionality to copy log segments, indexes to the target remote storage for each topic partition enabled with tiered storage. This involves creating scheduled tasks for all leader partition replicas to copy their log segments in sequence to tiered storage.

Reviewers: Jun Rao <junrao@gmail.com>, Luke Chen <showuon@gmail.com>
2023-04-12 13:55:36 +08:00
Victoria Xia 17b4569d70
KAFKA-14834: [2/N] Test coverage for out-of-order data in joins (#13497)
In preparation for updating DSL join processors to have updated semantics when versioned stores are used (cf KIP-914), this PR adds test coverage for out-of-order data in joins to the existing integration tests for stream-table joins and primary-key table-table joins. Follow-up PRs will build on top of this change by adding new tests for versioned stores, and the out-of-order data will produce different results in those settings.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2023-04-11 20:42:55 -07:00
José Armando García Sancio 672dd3ab6a
KAFKA-13020; Implement reading Snapshot log append timestamp (#13345)
The SnapshotReader exposes the "last contained log time". This is mainly used during snapshot cleanup. The previous implementation used the append time of the snapshot record. This is not accurate as this is the time when the snapshot was created and not the log append time of the last record included in the snapshot.

The log append time of the last record included in the snapshot is store in the header control record of the snapshot. The header control record is the first record of the snapshot.

To be able to read this record, this change extends the RecordsIterator to decode and expose the control records in the Records type.

Reviewers: Colin Patrick McCabe <cmccabe@apache.org>
2023-04-07 09:25:54 -07:00
Victoria Xia df59cc1a01
KAFKA-14491: [20/N] Add public-facing methods for versioned stores (#13442)
Until this PR, all the code added for KIP-889 for introducing versioned stores to Kafka Streams has been accessible from internal packages only. This PR exposes the stores via public Stores.java methods, and also updates the TopologyTestDriver.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2023-04-05 09:27:53 -07:00
Yash Mayya 970dea60e8
KAFKA-14785 (KIP-875): Connect offset read REST API (#13434)
Reviewers: Chris Egerton <chrise@aiven.io>
2023-04-02 13:09:33 -04:00
vamossagar12 c14f56b484
KAFKA-14586: Moving StreamResetter to tools (#13127)
Moves StreamResetter to tools project.

Reviewers: Federico Valeri <fedevaleri@gmail.com>, Christo Lolov <lolovc@amazon.com>, Bruno Cadonna <cadonna@apache.org>
2023-03-28 14:43:22 +02:00
David Arthur f1b3732fa6
KAFKA-14796 Migrate ACLs from AclAuthorizor to KRaft (#13368)
This patch refactors the loadCache method in AclAuthorizer to make it reusable by ZkMigrationClient.
The loaded ACLs are converted to AccessControlEntryRecord. I noticed we still have the defunct
AccessControlRecord, so I've deleted it.

Also included here are the methods to write ACL changes back to ZK while in dual-write mode.

Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Colin P. McCabe <cmccabe@apache.org>
2023-03-27 16:12:02 -07:00
Kirk True a3252629a3
KAFKA-14365: Extract common logic from Fetcher (#13425)
* KAFKA-14365: Extract common logic from Fetcher

Extract logic from Fetcher into AbstractFetcher.

Also introduce FetchConfig as a more concise way to delineate state from
incoming configuration.

Formalized the defaults in CommonClientConfigs and ConsumerConfig to be
accessible elsewhere.

* Removed overridden methods in favor of synchronizing where needed

Reviewers: Guozhang Wang <wangguoz@gmail.com>
2023-03-24 14:33:13 -07:00
Colin Patrick McCabe ed400e4c0d
KAFKA-14835: Create ControllerMetadataMetricsPublisher (#13438)
Separate out KRaft controller metrics into two groups: metrics directly managed by the
QuorumController, and metrics handled by an external publisher. This separation of concerns makes
the code easier to reason about, by clarifying what metrics can be changed where.

The external publisher, ControllerServerMetricsPublisher, handles all metrics which are related to
the content of metadata. For example, metrics about number of topics or number of partitions, etc.
etc. It fits into the MetadataLoader metadata publishing framework as another publisher.  Since
ControllerServerMetricsPublisher operates off of a MetadataImage, we don't have to create
(essentially) another copy of the metadata in memory, as ControllerMetricsManager. This reduces
memory consumption. Another benefit of operating off of the MetadataImage is that we don't have to
have special handling for each record type, like we do now in ControllerMetricsManager.

Reviewers: David Arthur <mumrah@gmail.com>
2023-03-24 11:26:53 -07:00
David Jacot 788cc11f45
KAFKA-14462; [3/N] Add `onNewMetadataImage` to `GroupCoordinator` interface (#13357)
The new group coordinator needs to access cluster metadata (e.g. topics, partitions, etc.) and it needs a mechanism to be notified when the metadata changes (e.g. to trigger a rebalance). In KRaft clusters, the easiest is to subscribe to metadata changes via the MetadataPublisher.

Reviewers: Justine Olshan <jolshan@confluent.io>
2023-03-08 08:52:01 +01:00
Proven Provenzano 38c409cf33
KAFKA-14084: SCRAM support in KRaft. (#13114)
This commit adds support to store the SCRAM credentials in a cluster with KRaft quorum servers and
no ZK cluster backing the metadata. This includes creating ScramControlManager in the controller,
and adding support for SCRAM to MetadataImage and MetadataDelta.

Change UserScramCredentialRecord to contain only a single tuple (name, mechanism, salt, pw, iter)
rather than a mapping between name and a list. This will avoid creating an excessively large record
if a single user has many entries. Because record ID 11 (UserScramCredentialRecord) has not been
used before, this is a compatible change. SCRAM will be supported in 3.5-IV0 and later.

This commit does not include KIP-900 SCRAM bootstrapping support, or updating the credential cache
on the controller (as opposed to broker). We will implement these in follow-on commits.

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
2023-03-03 10:23:34 -08:00
vamossagar12 bb3111f472
KAFKA-14580: Moving EndToEndLatency from core to tools module (#13095)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Federico Valeri <fedevaleri@gmail.com>, Ismael Juma <mlists@juma.me.uk>
2023-03-02 12:05:22 +01:00
Matthew Wong 8d32a0f246
[KAFKA-14685] Refactor logic to handle OFFSET_MOVED_TO_TIERED_STORAGE error (#13206)
Reviewers: Rittika Adhikari <rittika.adhikari@gmail.com>, Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>, Alexandre Dupriez <alexandre.dupriez@gmail.com>, Jun Rao <junrao@gmail.com>
2023-02-24 15:29:35 -08:00
Satish Duggana 069ce59e1e
KAFKA 14714: Move/Rewrite RollParams, LogAppendInfo, and LeaderHwChange to storage module. (#13255)
Reviewers: Jun Rao <junrao@gmail.com>, Ismael Juma <ismael@juma.me.uk>
2023-02-22 23:12:04 +05:30
Satish Duggana 322ac86ba2
KAFKA-14706: Move/rewrite ShutdownableThread to server-common module. (#13234)
Move/rewrite ShutdownableThread to server-common module.

Reviewers: Luke Chen <showuon@gmail.com>, Ismael Juma <ismael@juma.me.uk>
2023-02-17 11:51:17 +08:00
Greg Harris 958bc0601c
KAFKA-5756: Wait for concurrent source task offset flush to complete before starting next flush (#13208)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chris Egerton <chrise@aiven.io>
2023-02-15 21:29:20 -05:00
Ron Dagostino 631e6be3a0
KAFKA-14711: kafaka-metadata-quorum.sh does not honor --command-confi… (#13241)
…g option

https://github.com/apache/kafka/pull/12951 accidentally changed the behavior of the `kafaka-metadata-quorum.sh` CLI by making it silently ignore a `--command-config <filename>` properties file that exists. This was an undetected regression in the 3.4.0 release.  This patch fixes the issue such that any such specified file will be honored.

Reviewers: José Armando García Sancio <jsancio@apache.org>, Ismael Juma <ismael@juma.me.uk>
2023-02-13 18:33:20 -05:00
David Jacot 39962eeeb3
KAFKA-14513; Add broker side PartitionAssignor interface (#13202)
This patch adds the broker side `PartitionAssignor` interface as detailed in KIP-848. The interfaces differs a bit from the KIP in the following ways:
* The POJOs are not defined within the interface because the interface is to heavy like this.
* The interface is kept in the `group-coordinator` module for now. We don't want to have it out there until KIP-848 is ready to be released. We will move it to its final destination later.

Reviewers: Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>, Christo Lolov <lolovc@amazon.com>, Guozhang Wang <wangguoz@gmail.com>
2023-02-10 08:26:00 +01:00
David Arthur cb4d9d1abf
KAFKA-14668 Avoid unnecessary UMR during ZK migration (#13183)
Only send UMR to ZK brokers if the cluster metadata or topic metadata has changed.

Reviewers: Akhilesh C <akhileshchg@users.noreply.github.com>, Colin P. McCabe <cmccabe@apache.org>
2023-02-09 13:24:02 -05:00
Chris Egerton f93d5af839
KAFKA-15086, KAFKA-9981: Intra-cluster communication for Mirror Maker 2 (#13137)
Reviewers: Daniel Urban <durban@cloudera.com>, Greg Harris <greg.harris@aiven.io>, Viktor Somogyi-Vass <viktorsomogyi@gmail.com>, Mickael Maison <mickael.maison@gmail.com>
2023-02-09 10:50:07 -05:00
Satish Duggana 1d3fb76092
KAFKA-14688 Move package org.apache.kafka.server.log.internals to org.apache.kafka.storage.internals.log (#13213)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2023-02-08 09:22:42 +05:30
Satish Duggana da2e8dce71
KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module. (#13046)
KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

For broader context on this change, you may want to look at KAFKA-14470: Move log layer to the storage module

Reviewers: Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>, Alexandre Dupriez <alexandre.dupriez@gmail.com>
2023-02-07 15:37:23 +05:30
David Jacot 094e343f18
KAFKA-14678; Move `__consumer_offsets` records from `core` to `group-coordinator` (#13200)
This patch moves the current `__consumer_offsets` records from the `core` module to the new `group-coordinator` module.

Reviewers: Christo Lolov <lolovc@amazon.com>, Mickael Maison <mickael.maison@gmail.com>
2023-02-07 09:06:56 +01:00
Federico Valeri 50e0e3c257
KAFKA-14582: Move JmxTool to tools (#13136)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2023-02-02 11:23:26 +01:00
Federico Valeri 72cfc994f5
KAFKA-14628: Move CommandLineUtils and CommandDefaultOptions to tools (#13131)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Christo Lolov <christololov@gmail.com>, Sagar Rao <sagarmeansocean@gmail.com>
2023-01-26 20:06:09 +01:00
David Jacot 2e0a005dd4
KAFKA-14367; Add internal APIs to the new `GroupCoordinator` interface (#13112)
This patch migrates all the internal APIs of the current group coordinator to the new `GroupCoordinator` interface. It also makes the current implementation package private to ensure that it is not used anymore.

Reviewers: Justine Olshan <jolshan@confluent.io>
2023-01-20 08:38:21 +01:00
Colin Patrick McCabe 8478bbb589
KAFKA-14601: Improve exception handling in KafkaEventQueue #13089
If KafkaEventQueue gets an InterruptedException while waiting for a condition variable, it
currently exits immediately. Instead, it should complete the remaining events exceptionally and
then execute the cleanup event. This will allow us to finish any necessary cleanup steps.

In order to do this, we require the cleanup event to be provided when the queue is contructed,
rather than when it's being shut down.

Also, handle cases where Event#handleException itself throws an exception.

Remove timed shutdown from the event queue code since nobody was using it, and it adds complexity.

Add server-common/src/test/resources/test/log4j.properties since this gradle module somehow avoided
having a test log4j.properties up to this point.

Reviewers: David Arthur <mumrah@gmail.com>
2023-01-12 10:03:14 -08:00
Akhilesh C db49070760
KAFKA-14493: Introduce Zk to KRaft migration state machine STUBs in KRaft controller. (#12998)
This patch introduces a preliminary state machine that can be used by KRaft
controller to drive online migration from Zk to KRaft.

MigrationState -- Defines the states we can have while migration from Zk to
KRaft.

KRaftMigrationDriver -- Defines the state transitions, and events to handle
actions like controller change, metadata change, broker change and have
interfaces through which it claims Zk controllership, performs zk writes and
sends RPCs to ZkBrokers.

MigrationClient -- Interface that defines the functions used to claim and
relinquish Zk controllership, read to and write from Zk.

Co-authored-by: David Arthur <mumrah@gmail.com>
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2023-01-09 10:44:11 -08:00
Ismael Juma 96d9710c17
KAFKA-14478: Move LogConfig/CleanerConfig and related to storage module (#13049)
Additional notable changes to fix multiple dependency ordering issues:

* Moved `ConfigSynonym` to `server-common`
* Moved synonyms from `LogConfig` to `ServerTopicConfigSynonyms `
* Removed `LogConfigDef` `define` overrides and rely on
   `ServerTopicConfigSynonyms` instead.
* Moved `LogConfig.extractLogConfigMap` to `KafkaConfig`
* Consolidated relevant defaults from `KafkaConfig`/`LogConfig` in the latter
* Consolidate relevant config name definitions in `TopicConfig`
* Move `ThrottledReplicaListValidator` to `storage`

Reviewers: Satish Duggana <satishd@apache.org>, Mickael Maison <mickael.maison@gmail.com>
2023-01-04 02:42:52 -08:00
Ismael Juma e8232edd24
KAFKA-14477: Move LogValidator and related to storage module (#13012)
Also improved `LogValidatorTest` to cover a bug that was originally
only caught by `LogAppendTimeTest`.

For broader context on this change, please check:

* KAFKA-14470: Move log layer to storage module

Reviewers: Jun Rao <junrao@gmail.com>
2022-12-21 16:57:02 -08:00
José Armando García Sancio 44b3177a08
KAFKA-14457; Controller metrics should only expose committed data (#12994)
The controller metrics in the controllers has three problems. 1) the active controller exposes uncommitted data in the metrics. 2) the active controller doesn't update the metrics when the uncommitted data gets aborted. 3) the controller doesn't update the metrics when the entire state gets reset.

We fix these issues by only updating the metrics when processing committed metadata records and reset the metrics when the metadata state is reset.

This change adds a new type `ControllerMetricsManager` which processes committed metadata records and updates the metrics accordingly. This change also removes metrics updating responsibilities from the rest of the controller managers. 

Reviewers: Ron Dagostino <rdagostino@confluent.io>
2022-12-20 10:55:14 -08:00
Colin Patrick McCabe 29c09e2ca1
MINOR: ControllerServer should use the new metadata loader and snapshot generator (#12983)
This PR introduces the new metadata loader and snapshot generator. For the time being, they are
only used by the controller, but a PR for the broker will come soon.

The new metadata loader supports adding and removing publishers dynamically. (In contrast, the old
loader only supported adding a single publisher.) It also passes along more information about each
new image that is published. This information can be found in the LogDeltaManifest and
SnapshotManifest classes.

The new snapshot generator replaces the previous logic for generating snapshots in
QuorumController.java and associated classes. The new generator is intended to be shared between
the broker and the controller, so it is decoupled from both.

There are a few small changes to the old snapshot generator in this PR. Specifically, we move the
batch processing time and batch size metrics out of BrokerMetadataListener.scala and into
BrokerServerMetrics.scala.

Finally, fix a case where we are using 'is' rather than '==' for a numeric comparison in
snapshot_test.py.

Reviewers: David Arthur <mumrah@gmail.com>
2022-12-15 16:53:07 -08:00
Ismael Juma 88725669e7
MINOR: Move MetadataQuorumCommand from `core` to `tools` (#12951)
`core` should only be  used for legacy cli tools and tools that require
access to `core` classes instead of communicating via the kafka protocol
(typically by using the client classes).

Summary of changes:
1. Convert the command implementation and tests to Java and move it to
    the `tools` module.
2. Introduce mechanism to capture stdout and stderr from tests.
3. Change `kafka-metadata-quorum.sh` to point to the new command class.
4. Adjusted the test classpath of the `tools` module so that it supports tests
    that rely on the `@ClusterTests` annotation.
5. Improved error handling when an exception different from `TerseFailure` is
    thrown.
6. Changed `ToolsUtils` to avoid usage of arrays in favor of `List`.

Reviewers: dengziming <dengziming1993@gmail.com>
2022-12-09 09:22:58 -08:00
David Arthur d40561e90a
KAFKA-14427 ZK client support for migrations (#12946)
This patch adds support for reading and writing ZooKeeper metadata during a KIP-866 migration.

For reading metadata from ZK, methods from KafkaZkClient and ZkData are reused to ensure we are decoding the JSON consistently.

For writing metadata, we use a new multi-op transaction that ensures only a single controller is writing to ZK. This is similar to the existing multi-op transaction that KafkaController uses, but it also includes a check on the new "/migration" ZNode. The transaction consists of three operations:

* CheckOp on /controller_epoch
* SetDataOp on /migration with zkVersion
* CreateOp/SetDataOp/DeleteOp (the actual operation being applied)

In the case of a batch of operations (such as topic creation), only the final MultiOp has a SetDataOp on /migration while the other requests use a CheckOp (similar to /controller_epoch).

Reviewers: Colin Patrick McCabe <cmccabe@apache.org>, dengziming <dengziming1993@gmail.com>
2022-12-08 13:14:01 -05:00
Colin Patrick McCabe 100e874671
MINOR: Move dynamic config logic to DynamicConfigPublisher (#12958)
Split out the logic for applying dynamic configurations to a KafkaConfig object from
BrokerMetadataPublisher into a new class, DynamicConfigPublisher. This will allow the
ControllerServer to also run this code, in a follow-up change.

Create separate KafkaConfig objects in BrokerServer versus ControllerServer. This is necessary
because the controller will apply configuration changes as soon as its raft client catches up to
the high water mark, whereas the broker will wait for the active controller to acknowledge it has
caught up in a heartbeat response. So when running in combined mode, we want two separate
KafkaConfig objects that are changed at different times.

Minor changes: improve the error message when catching up broker metadata fails. Fix incorrect
indentation in checkstyle/import-control.xml. Invoke AppInfoParser.unregisterAppInfo from
SharedServer.stop so that it happens only when both the controller and broker have shut down.

Reviewers: David Arthur <mumrah@gmail.com>
2022-12-07 10:43:34 -08:00
Rajini Sivaram d23ce20bdf
KAFKA-14352: Rack-aware consumer partition assignment protocol changes (KIP-881) (#12954)
Reviewers: David Jacot <djacot@confluent.io>
2022-12-07 11:41:21 +00:00
Patrik Marton 1c10d107fe
KAFKA-14293: Basic Auth filter should set the SecurityContext after a successful login (#12846)
Reviewers: Greg Harris <greg.harris@aiven.io>, Chris Egerton <chrise@aiven.io>
2022-12-05 09:38:40 -05:00
Colin Patrick McCabe a3f5eb6e35
MINOR: Implement EventQueue#size and EventQueue#empty (#12930)
Implement functions to measure the number of events in the event queue.

Reviewers: David Arthur <mumrah@gmail.com>
2022-12-01 09:04:04 -08:00
David Jacot 98e19b3000
KAFKA-14367; Add `JoinGroup` to the new `GroupCoordinator` interface (#12845)
This patch adds `joinGroup` to the new `GroupCoordinator` interface and updates `KafkaApis` to use it.

For the context, I will do the same for all the other interactions with the current group coordinator. In order to limit the changes, I have chosen to introduce the `GroupCoordinatorAdapter` that translates the new interface to the old one. It is basically a wrapper. This allows keeping the current group coordinator untouched for now and focus on the `KafkaApis` changes. Eventually, we can remove `GroupCoordinatorAdapter`.

Reviewers: Justine Olshan <jolshan@confluent.io>, Jeff Kim <jeff.kim@confluent.io>, Luke Chen <showuon@gmail.com>, Jason Gustafson <jason@confluent.io>
2022-11-29 20:39:12 +01:00
Greg Harris 31c69ae932
KAFKA-14346: Remove hard-to-mock javax.crypto calls (#12866)
Reviewers: Chris Egerton <chrise@aiven.io>
2022-11-17 18:10:17 -05:00
Greg Harris fca5bfe13c
KAFKA-14346: Remove hard-to-mock RestClient calls (#12828)
Reviewers: Chris Egerton <chrise@aiven.io>
2022-11-17 17:51:54 -05:00
vamossagar12 9a793897ec
KAFKA-13152: KIP-770, cache size config deprecation (#12758)
PR implementing KIP-770 (#11424) was reverted as it brought in a regression wrt pausing/resuming the consumer. That KIP also introduced a change to deprecate config CACHE_MAX_BYTES_BUFFERING_CONFIG and replace it with STATESTORE_CACHE_MAX_BYTES_CONFIG.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2022-10-20 17:03:50 -07:00
Colin Patrick McCabe dac81161db
MINOR; Introduce ImageWriter and ImageWriterOptions (#12715)
This PR adds a new ImageWriter interface which replaces the generic Consumer interface which
accepted lists of records. It is better to do batching in the ImageWriter than to try to deal with
that complexity in the MetadataImage#write functions, especially since batching is not semantically
meaningful in KRaft snapshots. The new ImageWriter interface also supports freeze and close, which
more closely matches the semantics of the underlying Raft classes.

The PR also adds an ImageWriterOptions class which we can use to pass parameters to control how the
new image is written. Right now, the parameters that we are interested in are the target metadata
version (which may be more or less than the original image's version) and a handler function which
is invoked whenever metadata is lost due to the target version.

Convert over the MetadataImage#write function (and associated functions) to use the new ImageWriter
and ImageWriterOptions. In particular, we now have a way to handle metadata losses by invoking
ImageWriterOptions#handleLoss. This allows us to handle writing an image at a lower version, for
the first time. This support is still not enabled externally by this PR, though. That will come in
a future PR.

Get rid of the use of SOME_RECORD_TYPE.highestSupportedVersion() in several places. In general, we
do not want to "silently" change the version of a record that we output, just because a new version
was added. We should be explicit about what record version numbers we are outputting.

Implement ProducerIdsDelta#toString, to make debug logs look better.

Move MockRandom to the server-common package so that other internal broker packages can use it.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2022-10-13 09:56:19 -07:00
Chris Egerton 18e60cb000
KAFKA-12497: Skip periodic offset commits for failed source tasks (#10528)
Also moves the Streams LogCaptureAppender class into the clients module so that it can be used by both Streams and Connect.

Reviewers: Nigel Liang <nigel@nigelliang.com>, Kalpesh Patel <kpatel@confluent.io>, John Roesler <vvcephei@apache.org>, Tom Bentley <tbentley@redhat.com>
2022-10-13 10:15:42 -04:00
Alexandre Garnier 62914129c7
KAFKA-14099 - Fix request logging in connect (#12434)
Reviewers: Chris Egerton <chrise@aiven.io>
2022-10-12 10:28:55 -04:00
Jason Gustafson c5745d2845
MINOR: Add initial property tests for StandardAuthorizer (#12703)
In https://github.com/apache/kafka/pull/12695, we discovered a gap in our testing of `StandardAuthorizer`. We addressed the specific case that was failing, but I think we need to establish a better methodology for testing which incorporates randomized inputs. This patch is a start in that direction. We implement a few basic property tests using jqwik which focus on prefix searching. It catches the case from https://github.com/apache/kafka/pull/12695 prior to the fix. In the future, we can extend this to cover additional operation types, principal matching, etc.

Reviewers: David Arthur <mumrah@gmail.com>
2022-10-04 16:31:43 -07:00
Kirk True 8e43548175
KAFKA-13725: KIP-768 OAuth code mixes public and internal classes in same package (#12039)
* KAFKA-13725: KIP-768 OAuth code mixes public and internal classes in same package

Move classes into a sub-package of "internal" named "secured" that
matches the layout more closely of the "unsecured" package.

Replaces the concrete implementations in the former packages with
sub-classes of the new package layout and marks them as deprecated. If
anyone is already using the newer OAuth code, this should still work.

* Fix checkstyle and spotbugs violations

Co-authored-by: Kirk True <kirk@mustardgrain.com>

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2022-09-23 13:15:15 +05:30
Manikumar Reddy 5587c65fd3 MINOR: Add configurable max receive size for SASL authentication requests
This adds a new configuration `sasl.server.max.receive.size` that sets the maximum receive size for requests before and during authentication.

Reviewers: Tom Bentley <tbentley@redhat.com>, Mickael Maison <mickael.maison@gmail.com>

Co-authored-by: Manikumar Reddy <manikumar.reddy@gmail.com>
Co-authored-by: Mickael Maison <mickael.maison@gmail.com>
2022-09-21 20:58:33 +05:30
Colin Patrick McCabe b401fdaefb MINOR: Add more validation during KRPC deserialization
When deserializing KRPC (which is used for RPCs sent to Kafka, Kafka Metadata records, and some
    other things), check that we have at least N bytes remaining before allocating an array of size N.

    Remove DataInputStreamReadable since it was hard to make this class aware of how many bytes were
    remaining. Instead, when reading an individual record in the Raft layer, simply create a
    ByteBufferAccessor with a ByteBuffer containing just the bytes we're interested in.

    Add SimpleArraysMessageTest and ByteBufferAccessorTest. Also add some additional tests in
    RequestResponseTest.

    Reviewers: Tom Bentley <tbentley@redhat.com>, Mickael Maison <mickael.maison@gmail.com>, Colin McCabe <colin@cmccabe.xyz>

    Co-authored-by: Colin McCabe <colin@cmccabe.xyz>
    Co-authored-by: Manikumar Reddy <manikumar.reddy@gmail.com>
    Co-authored-by: Mickael Maison <mickael.maison@gmail.com>
2022-09-21 20:58:23 +05:30