Remove Apache ZooKeeper from the Apache Kafka build. Also remove commons IO, commons CLI, and netty, which were dependencies we took only because of ZooKeeper.
In order to keep the size of this PR manageable, I did not remove all classes which formerly interfaced with ZK. I just removed the ZK types. Fortunately, Kafka generally wrapped ZK data structures rather than using them directly.
Some classes were pretty entangled with ZK, so it was easier just to stub them out. For ZkNodeChangeNotificationListener.scala, PartitionStateMachine.scala, ReplicaStateMachine.scala, KafkaZkClient.scala, and ZookeeperClient.scala, I replaced all the functions with "throw new UnsupportedOperationException". Since the tests for these classes have been removed, as well as the ZK-based broker code, this should be OK as an incremental step.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
If ELR is enabled, we need to set a cluster-level min.insync.replicas, and remove all broker-level overrides. The reason for this is that if brokers disagree about which partitions are under min ISR, it breaks the KIP-966 replication invariants. In order to enforce this, when the eligible.leader.replicas.version feature is turned on, we automatically remove all broker-level min.insync.replicas overrides, and create the required cluster-level override if needed. Similarly, if the cluster was created with eligible.leader.replicas.version enabled, we create a similar cluster-level record. In both cases, we don't allow setting overrides for individual brokers afterwards, or removing the cluster-level override.
Split ActivationRecordsGeneratorTest up into multiple test cases rather than having it be one giant test case.
Fix a bug in QuorumControllerTestEnv where we would replay records manually on objects, racing with the active controller thread. Instead, we should simply ensure that the initial bootstrap records contains what we want.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
* KAFKA-18321: Add StreamsGroupMember, MemberState and Assignment classes
This commit adds the classes to represent a Streams group member in the
consumer coordinator.
Reviewers: Bill Bejeck <bill@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>
See https://issues.apache.org/jira/browse/KAFKA-18326 for more information. The main bug here is that in the old implementation, deleted cache entries would be skipped so long as they didn't equal the next store key, which resulted in potentially skipping tombstones for future keys in the store.
Reviewers: Guozhang Wang <guozhang.wang.us@gmail.com>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>
Looking at the [history](https://ge.apache.org/scans/tests?search.rootProjectNames=kafka&search.timeZoneId=Europe%2FZurich&tests.container=kafka.api.PlaintextAdminIntegrationTest&tests.test=testConsumerGroups(String%2C%20String)%5B2%5D), I found out that one source of flakiness is due to syncCommit failing with CommitFailedException. We can ignore it and retry on the next iteration.
```
[2025-01-07 10:17:00,783] ERROR [Consumer instanceId=test_instance_id_1, clientId=test_client_id, groupId=test_group_id] OffsetCommit failed for member VfImExrxT3-w_HNJcTkqnw with stale member epoch error. Last epoch sent: 2 (org.apache.kafka.clients.consumer.internals.CommitRequestManager:773)
Exception in thread "Thread-6" org.apache.kafka.clients.consumer.CommitFailedException: OffsetCommit failed with stale member epoch.The member epoch is stale. The member must retry after receiving its updated member epoch via the ConsumerGroupHeartbeat API.
at org.apache.kafka.clients.consumer.internals.CommitRequestManager.commitSyncExceptionForError(CommitRequestManager.java:481)
at org.apache.kafka.clients.consumer.internals.CommitRequestManager.lambda$commitSyncWithRetries$7(CommitRequestManager.java:472)
at java.base/java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:863)
at java.base/java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:841)
at java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:510)
at java.base/java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:2162)
at org.apache.kafka.clients.consumer.internals.CommitRequestManager$OffsetCommitRequestState.onResponse(CommitRequestManager.java:776)
at org.apache.kafka.clients.consumer.internals.CommitRequestManager$RetriableRequestState.handleClientResponse(CommitRequestManager.java:893)
at org.apache.kafka.clients.consumer.internals.CommitRequestManager$RetriableRequestState.lambda$buildRequestWithResponseHandling$0(CommitRequestManager.java:883)
at java.base/java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:863)
at java.base/java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:841)
at java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:510)
at java.base/java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:2147)
at org.apache.kafka.clients.consumer.internals.NetworkClientDelegate$FutureCompletionHandler.onComplete(NetworkClientDelegate.java:433)
at org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:154)
at org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:669)
at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:661)
at org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.poll(NetworkClientDelegate.java:153)
at org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread.runOnce(ConsumerNetworkThread.java:160)
at org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread.run(ConsumerNetworkThread.java:106)
```
Reviewers: Lianet Magrans <lmagrans@confluent.io>
Remove RaftManager.maybeDeleteMetadataLogDir since it was only used during ZK migration, and that code has been removed.
Similarly, remove RaftManagerTest.testKRaftBrokerDoesNotDeleteMetadataLog which tested that function.
Remove AutoTopicCreationManagerTest since it tests the ZK-mode-only AutoTopicReationManager.
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
30 sec is more inline with the default request.timeout.ms and is still very acceptable for a CLI interaction in my opinion
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Following https://github.com/apache/kafka/pull/18261, this patch updates the Share Coordinator to use the new record format.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Andrew Schofield <aschofield@confluent.io>
The consumer/producer JavaDocs still contain instruction for naively
computing the offset to be committed.
This PR updates the JavaDocs with regard to the improvements of KIP-1094.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, Lianet Magrans <lmagrans@confluent.io>
There are times when the controller has a high event processing time, such as during startup, or when creating a topic with many partitions. We can see these processing times in the p99 metric (kafka.controller:type=ControllerEventManager,name=EventQueueProcessingTimeMs), however it's difficult to see exactly which event is causing high processing time.
With DEBUG logs, we see every event along with its processing time. Even with this, it's a bit tedious to find the event with a high processing time.
This PR logs all events which take longer than 2 seconds at ERROR level. This will help identify events that are taking far too long, and which could be disruptive to the operation of the controller. The slow event logging looks like this:
```
[2024-12-20 15:03:39,754] ERROR [QuorumController id=1] Exceptionally slow controller event createTopics took 5240 ms. (org.apache.kafka.controller.EventPerformanceMonitor)
```
Also, every 60 seconds, it logs some event time statistics, including average time, maximum time, and the name of the event which took the longest. This periodic message looks like this:
```
[2024-12-20 15:35:04,798] INFO [QuorumController id=1] In the last 60000 ms period, 333 events were completed, which took an average of 12.34 ms each. The slowest event was handleCommit[baseOffset=0], which took 41.90 ms. (org.apache.kafka.controller.EventPerformanceMonitor)
```
An operator can disable these logs by adding the following to their log4j config:
```
org.apache.kafka.controller.EventPerformanceMonitor=OFF
```
Reviewers: Colin P. McCabe <cmccabe@apache.org>
A simplified port of "ChangelogTopics" from the client-side to the group coordinator
Compared to the client-side version, the implementation uses immutable data structures, and returns the computed number of partitions instead of modifying mutable data structures and calling the admin client.
Reviewers: Bruno Cadonna <cadonna@apache.org>