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>
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>
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>
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>
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>
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>
This patch exposes the group coordinator config `CONSUMER_GROUP_MIGRATION_POLICY_CONFIG`.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <djacot@confluent.io>
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>
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>
This patch disallows enabling the new consumer rebalance protocol in ZK mode.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Justine Olshan <jolshan@confluent.io>
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>
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>
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>
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>
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>
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>
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>
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>
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>
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>
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>
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.
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
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>
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>
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>
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>