Commit Graph

4818 Commits

Author SHA1 Message Date
Andrew Schofield 8f82f14a48
KAFKA-16713: Define initial set of RPCs for KIP-932 (#16022)
This PR defines the initial set of RPCs for KIP-932. The RPCs for the admin client and state management are not in this PR.

Reviewers: Apoorv Mittal <amittal@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>
2024-06-03 11:52:35 +05:30
Ken Huang 8507693229
KAFKA-16859 Cleanup check if tiered storage is enabled (#16153)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-03 11:04:58 +08:00
Ken Huang 2c82ecd67f
KAFKA-16807 DescribeLogDirsResponseData#results#topics have unexpected topics having empty partitions (#16042)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-02 17:33:02 +08:00
Colin Patrick McCabe 8ace33b47f
KAFKA-16757: Fix broker re-registration issues around MV 3.7-IV2 (#15945)
When upgrading from a MetadataVersion older than 3.7-IV2, we need to resend the broker registration, so that the controller can record the storage directories. The current code for doing this has several problems, however. One is that it tends to trigger even in cases where we don't actually need it. Another is that when re-registering the broker, the broker is marked as fenced.

This PR moves the handling of the re-registration case out of BrokerMetadataPublisher and into BrokerRegistrationTracker. The re-registration code there will only trigger in the case where the broker sees an existing registration for itself with no directories set.  This is much more targetted than the original code.

Additionally, in ClusterControlManager, when re-registering the same broker, we now preserve its fencing and shutdown state, rather than clearing those. (There isn't any good reason re-registering the same broker should clear these things... this was purely an oversight.) Note that we can tell the broker is "the same" because it has the same IncarnationId.

Reviewers: Gaurav Narula <gaurav_narula2@apple.com>, Igor Soarez <soarez@apple.com>
2024-06-01 23:51:39 +01:00
Chia Chuan Yu e33eb82fed
KAFKA-16574 The metrics of LogCleaner disappear after reconfiguration (#15863)
Reviewers: Gaurav Narula <gaurav_narula2@apple.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-06-02 01:02:03 +08:00
TaiJuWu db2a09fa90
KAFKA-16652 add unit test for ClusterTemplate offering zero ClusterConfig (#15862)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-02 00:56:54 +08:00
David Jacot ba61ff0cd9
KAFKA-16860; [1/2] Introduce group.version feature flag (#16120)
This patch introduces the `group.version` feature flag with one version:
1) Version 1 enables the new consumer group rebalance protocol (KIP-848).

Reviewers: Justine Olshan <jolshan@confluent.io>
2024-05-31 12:48:55 -07:00
Kamal Chandraprakash cdd4455cb8
KAFKA-16866 Used the right constant in RemoteLogManagerTest#testFetchQuotaManagerConfig (#16152)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-01 01:14:31 +08:00
Mickael Maison b6d0fb055d
MINOR: Refactor DynamicConfig (#16133)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-01 01:09:46 +08:00
Ken Huang 21caf6b123
KAFKA-16629 Add broker-related tests to ConfigCommandIntegrationTest (#15840)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-31 20:24:33 +08:00
Kuan-Po (Cooper) Tseng 3d125a2322
MINOR: Add more unit tests to LogSegments (#16085)
add more unit tests to LogSegments and do some small refactor in LogSegments.java

Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-05-31 16:07:38 +08:00
Chia-Ping Tsai b0fb2ac06d
KAFKA-16866 RemoteLogManagerTest.testCopyQuotaManagerConfig failing (#16146)
Reviewers: Justine Olshan <jolshan@confluent.io>, Satish Duggana <satishd@apache.org>
2024-05-31 06:32:50 +05:30
Justine Olshan 7c1bb1585f
KAFKA-16308 [2/N]: Allow unstable feature versions and rename unstable metadata config (#16130)
As per KIP-1022, we will rename the unstable metadata versions enabled config to support all feature versions.

Features is also updated to return latest production and latest testing versions of each feature.

A feature is production ready when the corresponding metadata version (bootstrapMetadataVersion) is production ready.

Adds tests for the feature usage of the unstableFeatureVersionsEnabled config

Reviewers: David Jacot <djacot@confluent.io>, Jun Rao <junrao@gmail.com>
2024-05-30 14:52:50 -07:00
David Jacot cd750582c0
MINOR: Enable transaction verification with new group coordinator in TransactionsTest (#16139)
While working on https://github.com/apache/kafka/pull/16120, I noticed that the transaction verification feature is disabled in `TransactionsTest` when the new group coordinator is enabled. We did this initially because the feature was not available in the new group coordinator but we fixed it a long time ago. We can enable it now.

Reviewers: Justine Olshan <jolshan@confluent.io>
2024-05-30 12:35:29 -07:00
Dongnuo Lyu a626e87303
MINOR: Make public the consumer group migration policy config
This patch exposes the group coordinator config `CONSUMER_GROUP_MIGRATION_POLICY_CONFIG`.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <djacot@confluent.io>
2024-05-30 11:36:11 -07:00
Krishna Agarwal bb6a042e99
KAFKA-16827: Integrate kafka native-image with system tests (#16046)
This PR does following things

System tests should bring up Kafka broker in the native mode
System tests should run on Kafka broker in native mode
Extract out native build command so that it can be reused.
Allow system tests to run on Native Kafka broker using Docker mechanism

To run system tests by bringing up Kafka in native mode:
Pass kafka_mode as native in the ducktape globals:--globals '{\"kafka_mode\":\"native\"}'

Running system tests by bringing up kafka in native mode via docker mechanism
_DUCKTAPE_OPTIONS="--globals '{\"kafka_mode\":\"native\"}'" TC_PATHS="tests/kafkatest/tests/"  bash tests/docker/run_tests.sh

To only bring up ducker nodes to cater native kafka
bash tests/docker/ducker-ak up -m native

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2024-05-30 22:24:23 +05:30
Abhijeet Kumar bb7db87f98
KAFKA-15265: Add Remote Log Manager quota manager (#15625)
Added the implementation of the quota manager that will be used to throttle copy and fetch requests from the remote storage. Reference KIP-956

Reviewers: Luke Chen <showuon@gmail.com>, Kamal Chandraprakash <kchandraprakash@uber.com>, Jun Rao <junrao@gmail.com>
2024-05-30 09:06:49 -07:00
Mickael Maison 8068a086a3
MINOR: Remove KafkaConfig dependency in KafkaRequestHandler (#16108)
Reviewers: Luke Chen <showuon@gmail.com>, Apoorv Mittal <amittal@confluent.io>
2024-05-30 11:51:24 +02:00
David Jacot 2a6078a4ce
MINOR: Prevent consumer protocol to be used in ZK mode (#16121)
This patch disallows enabling the new consumer rebalance protocol in ZK mode.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Justine Olshan <jolshan@confluent.io>
2024-05-29 23:02:21 -07:00
Murali Basani 3d14690cbf
KAFKA-16790: Update RemoteLogManager configuration in broker server (#16005)
n BrokerServer.scala, brokerMetadataPublishers are configured and when there are metadata updates remoteLogManager is not configured by then.
Ex : remoteLogManager.foreach(rlm => rlm.onLeadershipChange(partitionsBecomeLeader.asJava, partitionsBecomeFollower.asJava, topicIds)) in ReplicaManager is invoked after publishers are instantiated, and here rlm has relevant managers configured.

This change makes sure rlm is configured before the brokerMetadataPublishers initialization.

Reviewers: Luke Chen <showuon@gmail.com>, Nikhil Ramakrishnan <nikrmk@amazon.com>
2024-05-30 08:21:30 +08:00
Justine Olshan 5e3df22095
KAFKA-16308 [1/N]: Create FeatureVersion interface and add `--feature` flag and handling to StorageTool (#15685)
As part of KIP-1022, I have created an interface for all the new features to be used when parsing the command line arguments, doing validations, getting default versions, etc.

I've also added the --feature flag to the storage tool to show how it will be used.

Created a TestFeatureVersion to show an implementation of the interface (besides MetadataVersion which is unique) and added tests using this new test feature.

I will add the unstable config and tests in a followup.

Reviewers: David Mao <dmao@confluent.io>, David Jacot <djacot@confluent.io>, Artem Livshits <alivshits@confluent.io>, Jun Rao <junrao@apache.org>
2024-05-29 16:36:06 -07:00
Mickael Maison 3f3f3ac155
MINOR: Delete KafkaSecurityConfigs class (#16113)
Reviewers: Omnia Ibrahim <o.g.h.ibrahim@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-05-30 05:55:24 +08:00
gongxuanzhang 0f0c9ecbf3
KAFKA-16771 First log directory printed twice when formatting storage (#16010)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-30 01:08:17 +08:00
Andrew Schofield 2d9994e0de
KAFKA-16722: Introduce ConsumerGroupPartitionAssignor interface (#15998)
KIP-932 introduces share groups to go alongside consumer groups. Both kinds of group use server-side assignors but it is unlikely that a single assignor class would be suitable for both. As a result, the KIP introduces specific interfaces for consumer group and share group partition assignors.

This PR introduces only the consumer group interface, `o.a.k.coordinator.group.assignor.ConsumerGroupPartitionAssignor`. The share group interface will come in a later release. The existing implementations of the general `PartitionAssignor` interface have been changed to implement `ConsumerGroupPartitionAssignor` instead and all other code changes are just propagating the change throughout the codebase.

Note that the code in the group coordinator that actually calculates assignments uses the general `PartitionAssignor` interface so that it can be used with both kinds of group, even though the assignors themselves are specific.

Reviewers: Apoorv Mittal <amittal@confluent.io>, David Jacot <djacot@confluent.io>
2024-05-29 08:31:52 -07:00
gongxuanzhang 0b75cf7c0b
KAFKA-16705 the flag "started" of RaftClusterInstance is false even though the cluster is started (#15946)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-29 22:38:00 +08:00
Luke Chen 897cab2a61
KAFKA-16399: Add JBOD support in tiered storage (#15690)
After JBOD is supported in KRaft, we should also enable JBOD support in tiered storage. Unit tests and Integration tests are also added.

Reviewers: Satish Duggana <satishd@apache.org>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Igor Soarez <soarez@apple.com>, Mickael Maison <mickael.maison@gmail.com>
2024-05-29 15:30:18 +08:00
PoAn Yang 4d04eb83ea
KAFKA-16796 Introduce new org.apache.kafka.tools.api.Decoder to replace kafka.serializer.Decoder (#16064)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-29 03:13:33 +08:00
Luke Chen a649bc457f
KAFKA-16711: Make sure to update highestOffsetInRemoteStorage after log dir change (#15947)
Reviewers: Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Satish Duggana <satishd@apache.org>
2024-05-28 21:35:49 +05:30
Omnia Ibrahim 64f699aeea
KAFKA-15853: Move general configs out of KafkaConfig (#16040)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-05-28 16:22:54 +02:00
Sanskar Jhajharia 699438b7f7
MINOR: Fix the config name in ProducerFailureHandlingTest (#16099)
When moving from KafkaConfig.ReplicaFetchMaxBytesProp we used ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG instead of ReplicationConfigs.REPLICA_FETCH_MAX_BYTES_CONFIG. This PR patches the same.

Reviewers: Omnia Ibrahim <o.g.h.ibrahim@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
2024-05-28 16:34:44 +05:30
Luke Chen 91284d8d7b
KAFKA-16709: abortAndPauseCleaning only when future log is not existed (#15951)
When doing alter replica logDirs, we'll create a future log and pause log cleaning for the partition( here). And this log cleaning pausing will resume after alter replica logDirs completes (here). And when in the resuming log cleaning, we'll decrement 1 for the LogCleaningPaused count. Once the count reached 0, the cleaning pause is really resuming. (here). For more explanation about the logCleaningPaused state can check here.

But, there's still one factor that could increase the LogCleaningPaused count: leadership change (here). When there's a leadership change, we'll check if there's a future log in this partition, if so, we'll create future log and pauseCleaning (LogCleaningPaused count + 1). So, if during the alter replica logDirs:

1. alter replica logDirs for tp0 triggered (LogCleaningPaused count = 1)
2. tp0 leadership changed (LogCleaningPaused count = 2)
3. alter replica logDirs completes, resuming logCleaning (LogCleaningPaused count = 1)
4. LogCleaning keeps paused because the count is always >  0

This PR fixes this issue by only abortAndPauseCleaning when future log is not existed. We did the same check in alterReplicaLogDirs. So this change can make sure there's only 1 abortAndPauseCleaning for either abortAndPauseCleaning or maybeAddLogDirFetchers. Tests also added.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Igor Soarez <soarez@apple.com>
2024-05-28 12:23:34 +08:00
Colin P. McCabe bac8df56ff MINOR: fix typo in KAFKA-16515 2024-05-27 08:53:53 -07:00
David Jacot da3304ecb6
KAFKA-16371; fix lingering pending commit when handling OFFSET_METADATA_TOO_LARGE (#16072)
This patch was initially created in #15536.

When there is a commit for multiple topic partitions and some, but not all, exceed the offset metadata limit, the pending commit is not properly cleaned up leading to UNSTABLE_OFFSET_COMMIT errors when trying to fetch the offsets with read_committed. This change makes it so the invalid commits are not added to the pendingOffsetCommits set.

Co-authored-by: Kyle Phelps <kyle.phelps@datadoghq.com>

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Justine Olshan <jolshan@confluent.io>
2024-05-27 08:10:37 -07:00
Kamal Chandraprakash 524ad1e14b
KAFKA-16452: Don't throw OOORE when converting the offset to metadata (#15825)
Don't throw OFFSET_OUT_OF_RANGE error when converting the offset to metadata, and next time the leader should increment the high watermark by itself after receiving fetch requests from followers. This can happen when checkpoint files are missing and being elected as a leader. 

Reviewers: Luke Chen <showuon@gmail.com>, Jun Rao <junrao@apache.org>
2024-05-27 17:44:23 +08:00
Colin P. McCabe 4f55786a8a KAFKA-16515: Fix the ZK Metadata cache confusion between brokers and controllers
ZkMetadataCache could theoretically return KRaft controller information from a call to
ZkMetadataCache.getAliveBrokerNode, which doesn't make sense. KRaft controllers are not part of the
set of brokers. The only use-case for this functionality was in MetadataCacheControllerNodeProvider
during ZK migration, where it allowed ZK brokers in migration mode to forward requests to
kcontrollers when appropriate. This PR changes MetadataCacheControllerNodeProvider to simply
delegate to quorumControllerNodeProvider in this case.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2024-05-24 10:16:59 -07:00
Colin P. McCabe 90892ae99f KAFKA-16516: Fix the controller node provider for broker to control channel
Fix the code in the RaftControllerNodeProvider to query RaftManager to find Node information,
rather than consulting a static map. Add a RaftManager.voterNode function to supply this
information. In KRaftClusterTest, add testControllerFailover to get more coverage of controller
failovers.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2024-05-24 09:52:47 -07:00
Gantigmaa Selenge c5cd190818
MINOR: Refactor SSL/SASL admin integration tests to not use a custom authorizer (#15377)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-05-24 12:50:47 +02:00
Viktor Somogyi-Vass 5a4898450d
KAFKA-15649: Handle directory failure timeout (#15697)
A broker that is unable to communicate with the controller will shut down
after the configurable log.dir.failure.timeout.ms.

The implementation adds a new event to the Kafka EventQueue. This event
is deferred by the configured timeout and will execute the shutdown
if the heartbeat communication containing the failed log dir is still
pending with the controller.

Reviewers: Igor Soarez <soarez@apple.com>
2024-05-23 16:36:39 +01:00
Mickael Maison ab0cc72499
MINOR: Move parseCsvList to server-common (#16029)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-23 16:01:45 +02:00
Mickael Maison e4e1116156
MINOR: Move Throttler to storage module (#16023)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-22 18:47:31 +02:00
Krishna Agarwal 271c04bd17
KAFKA-15444: Native docker image for Apache Kafka (KIP-974) (#15927)
This PR aims to add Docker Image for GraalVM based Native Kafka Broker as per the following KIP - https://cwiki.apache.org/confluence/display/KAFKA/KIP-974%3A+Docker+Image+for+GraalVM+based+Native+Kafka+Broker

This PR adds the following functionalities:

Ability to build the docker image for Native Apache Kafka
   - Dockerfile
   - Launch script
   - metadata configs required by graalVM native-image: link
Add Kafka startup ability in the KafkaDockerWrapper.scala
Ability to build and test the image - integrated with the existing JVM docker image framework.
2024-05-22 10:52:46 +05:30
Mickael Maison affe8da54c
KAFKA-7632: Support Compression Levels (KIP-390) (#15516)
Reviewers: Jun Rao <jun@confluent.io>,  Luke Chen <showuon@gmail.com>
Co-authored-by: Lee Dongjin <dongjin@apache.org>
2024-05-21 17:58:49 +02:00
TaiJuWu 89083520ef
KAFKA-16654 Refactor kafka.test.annotation.Type and ClusterTestExtensions (#15916)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-21 22:29:06 +08:00
Nikhil Ramakrishnan b5a013e456 KAFKA-16513; Add test for WriteTxnMarkers with AlterCluster permission
In #15837, we introduced the change to allow calling the WriteTxnMarkers API with AlterCluster permissions. This PR proposes 2 enhancements:

- When a WriteTxnMarkers request is received, it is first authorized against the alter cluster permission. If the user does not have this permission, a 'deny' will be logged. However, if the user does have the cluster action permission, the request will be successfully authorized.  Don't log the first deny to avoid confusion.
- Add a `WriteTxnMarkersRequest` to be called from the test `testAuthorizationWithTopicExisting`, so that the request can be exercised and verified with both possible permissions.

Author: Nikhil Ramakrishnan <nikrmk@amazon.com>

Reviewers: Christo Lolov <lolovc@amazon.com>

Closes #15952 from nikramakrishnan/kip1037-addTest
2024-05-21 10:34:28 +01:00
Lianet Magrans 52b4596dae
KAFKA-16675: Refactored and new rebalance callbacks integration tests (#15965)
Move existing rebalance callback + consumer.position test to the PlaintextConsumerCallbackTest file (refactored to reuse the new helper funcs available)
Add new integration tests for callbacks interaction with seek and pause
Minor cleanup in the callbacks test file

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2024-05-21 10:40:57 +02:00
Nikolay c10bb58d1c
KAFKA-14588 [4/N] ConfigCommandTest rewritten in java (#15839)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-21 16:39:39 +08:00
David Jacot b4c2d66801
KAFKA-16770; [1/N] Coalesce records into bigger batches (#15964)
We have discovered during large scale performance tests that the current write path of the new coordinator does not scale well. The issue is that each write operation writes synchronously from the coordinator threads. Coalescing records into bigger batches helps drastically because it amortizes the cost of writes. Aligning the batches with the snapshots of the timelines data structures also reduces the number of in-flight snapshots.

This patch is the first of a series of patches that will bring records coalescing into the coordinator runtime. As a first step, we had to rework the PartitionWriter interface and move the logic to build MemoryRecords from it to the CoordinatorRuntime. The main changes are in these two classes. The others are related mechanical changes.

Reviewers: Justine Olshan <jolshan@confluent.io>
2024-05-20 23:47:09 -07:00
Gaurav Narula 95adb7bfbf
MINOR: ensure KafkaServerTestHarness::tearDown is always invoked (#15996)
An exception thrown while closing the client instances in `IntegrationTestHarness::tearDown` may result in `KafkaServerTestHarness::tearDown` not being invoked. This would result in thread leaks of the broker and controller threads spawned in the failing test.

An example of this is the [CI run](https://ci-builds.apache.org/blue/organizations/jenkins/Kafka%2Fkafka-pr/detail/PR-15994/1/tests) for #15994 where `Build / JDK 8 and Scala 2.12 / testCoordinatorFailover(String, String).quorum=kraft+kip848.groupProtocol=consumer – kafka.api.PlaintextConsumerTest` failing results in `consumers.foreach(_.close(Duration.ZERO))` in `IntegrationTestHarness::tearDown` throwing an exception.

A side effect of this is it poisons Gradle test runner JVM and prevents tests in other unrelated classes from executing as `@BeforeAll` check in QuorumTestHarness would cause them to fail immediately.

This PR encloses the client closure in try-finally to ensure`KafkaServerTestHarness::tearDown` is always invoked.

Reviewers: Nikhil Ramakrishnan <nikrmk@amazon.com>, Igor Soarez <soarez@apple.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-05-21 03:10:08 +08:00
Gaurav Narula 412b05df00
KAFKA-16789 Fix thread leak detection for event handler threads (#15984)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-19 18:21:56 +08:00
Justine Olshan 3e15ab98ec
KAFKA-16992: InvalidRequestException: ADD_PARTITIONS_TO_TXN with version 4 which is not enabled when upgrading from kafka (#15971)
We weren't enabling discoverBrokerVersions to check the supported versions in the AddPartitionsToTxnManager. This means that any verification request (or any AddPartitionsToTxnRequest version) from a newer broker would fail when sending to an older broker.

The bulk of this change is adding additional transactions system tests for old versions.
One test upgrades the cluster completely. This didn't catch the issue but could be useful.

The other test forces a new broker to send a verification request to an older one. Without the discoverBrokerVersions change, all tests between mixed brokers failed. (We introduced a new request version in 3.8 -- which is a separate version from the one that caused the bug for 3.5 -> 3.6) With the addition, the tests all passed.

I also manually ran a test for 3.5 -> 3.6 since the issue there was slightly different and was caused by the unstableLatestVersion flag being enabled. This change should fix this as well. 👍

Reviewers:  David Jacot <djacot@confluent.io>
2024-05-17 21:35:28 -07:00