Commit Graph

4818 Commits

Author SHA1 Message Date
Omnia Ibrahim e1bfaec49d
KAFKA-15853 Move metrics configs out of KafkaConfig (#15822)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-30 01:19:05 +08:00
Kuan-Po (Cooper) Tseng 5de5d967ad
KAFKA-16560 Refactor/cleanup BrokerNode/ControllerNode/ClusterConfig (#15761)
* Make ClusterConfig immutable
* Make BrokerNode immutable
* Refactor out build argument in ControllerNode
* Add setPrefix and replace put property with set map in ClusterConfig
* Remove rollingBrokerRestart from ClusterInstance interface
* Refactor KRaftClusterTest#doOnStartedKafkaCluster

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-28 02:00:56 +08:00
TaiJuWu 4060d4370e
KAFKA-6527 Enable DynamicBrokerReconfigurationTest.testDefaultTopicConfig (#15796)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-27 08:00:29 +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
Gaurav Narula 025f9816f1
MINOR: fix javadoc warnings (#15527)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-26 08:31:52 +08:00
Omnia Ibrahim 6feae817d2
MINOR: Rename RaftConfig to QuorumConfig (#15797)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-26 03:08:31 +08:00
TaiJuWu ce9026f597
MINOR: Modified System.getProperty("line.separator") to java.lang.System.lineSeparator() (#15782)
Reviewers: Igor Soarez  <soarez@apple.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-26 02:32:11 +08:00
Mickael Maison 0a6d5ff23c
MINOR: Various cleanups in core (#15786)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Omnia Ibrahim <o.g.h.ibrahim@gmail.com>
2024-04-25 16:45:00 +02:00
TingIāu "Ting" Kì 864744ffd4
KAFKA-16610 Replace "Map#entrySet#forEach" by "Map#forEach" (#15795)
Reviewers: Apoorv Mittal <amittal@confluent.io>, Igor Soarez <soarez@apple.com>
2024-04-25 01:52:24 +01:00
PoAn Yang 81c222e977
KAFKA-16613 remove TestUtils#subscribeAndWaitForRecords (#15794)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-25 02:15:21 +08:00
Kamal Chandraprakash a8c0f2b98f
KAFKA-16605: Fix the flaky LogCleanerParameterizedIntegrationTest (#15787)
Even if the log start offset is updated, the log deletion might still not completed. Making the test more robust.

Reviewers: Luke Chen <showuon@gmail.com>
2024-04-24 17:52:52 +08:00
PoAn Yang a38185280c
KAFKA-16424: remove truncated logs after alter dir (#15616)
If there are some logs to be deleted during the log dir movement, we'll send for a scheduler to do the deletion later.
However, when the log dir movement completed, the future log is renamed, the async log deletion will fail with no file existed error.

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

Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, SoontaekLim <soontaek.lim@neya.kr>, Johnny Hsu <johnnyhsu@fb.com>
2024-04-24 17:51:29 +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
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
Ken Huang fb529d8966
KAFKA-16548 Avoid decompressing/collecting all records when all we want to do is to find a single matched record from remote storage (#15765)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-22 21:23:11 +08:00
charliecheng630 1763fe19dd
KAFKA-16549 suppress the warnings from RemoteLogManager (#15767)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-22 20:42:17 +08:00
Lucas Brutschy ed47e37b28
KAFKA-16103: AsyncConsumer should await pending async commits on commitSync and close (#15613)
The javadoc for KafkaConsumer.commitSync says:

Note that asynchronous offset commits sent previously with the {@link #commitAsync(OffsetCommitCallback)}
(or similar) are guaranteed to have their callbacks invoked prior to completion of this method.

This is not always true in the async consumer, where there is no code at all to make sure that the callback is executed before commitSync returns.

Similarly, the async consumer is also missing logic to await callback execution in close. While the javadoc doesn't explicitly promise callback execution, it promises "completing commits", which one would reasonably expect to include callback execution. Also, the legacy consumer contains some code to execute callbacks before closing.

This change proposed a number of fixes to clean up the callback execution guarantees in the async consumer:

We keep track of the incomplete async commit
futures and wait for them to complete before returning from
commitSync or close (if there is time).
Since we need to block to make sure that our previous commits are
completed, we allow the consumer to wake up.
Some similar gaps are addressed in the legacy consumer, see #15693

Testing
Two new integration tests and a couple of unit tests.

Reviewers: Bruno Cadonna <cadonna@apache.org>, Kirk True <ktrue@confluent.io>, Lianet Magrans <lianetmr@gmail.com>
2024-04-22 13:26:15 +02:00
Omnia Ibrahim 5e96e5c898
KAFKA-15853 Refactor KafkaConfig to use PasswordEncoderConfigs (#15770)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-22 00:47:57 +08:00
TingIāu "Ting" Kì 98548c517d
KAFKA-16591 Clear all admins after close all them. (#15763)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-21 06:26:44 +08:00
Kuan-Po (Cooper) Tseng ced79ee12f
KAFKA-16552 Create an internal config to control InitialTaskDelayMs in LogManager to speed up tests (#15719)
Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-20 20:34:02 +08:00
hudeqi 613d4c8578
MINOR: fix hint in fetchOffsetForTimestamp (#15757)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-20 20:14:21 +08:00
Gaurav Narula 0308543d59
KAFKA-16082 Avoid resuming future replica if current replica is in the same directory (#15136)
It is observed that for scenario (3), i.e. a broker crashes while it
waits for the future replica to catch up for the second time and the
`dir1` is unavailable when the broker is restarted, the
broker tries to create the partition in `dir2` according to the metadata
in the controller. However, ReplicaManager also tries to resume the
stale future replica which was abandoned when the broker crashed. This
results in the renaming of the future replica to fail eventually because
the directory for the topic partition already exists in `dir2` and the
broker then marks `dir2` as offline.

This PR attempts to fix this behaviour by ignoring any future replicas
which are in the same directory as where the log exists. It further
marks the stale future replica for deletion.

Reviewers: Omnia Ibrahim <o.g.h.ibrahim@gmail.com>,  Igor Soarez <soarez@apple.com>, Proven Provenzano <pprovenzano@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-20 04:18:51 +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
Calvin Liu 53ff1a5a58
KAFKA-15585: DescribeTopicPartitions client side change. (#15470)
Add the support for DescribeTopicPartitions API to AdminClient. For this initial implementation, we are simply loading all of the results into memory on the client side. 

Reviewers: Andrew Schofield <aschofield@confluent.io>, Kirk True <ktrue@confluent.io>, David Jacot <djacot@confluent.io>, Artem Livshits <alivshits@confluent.io>, David Arthur <mumrah@gmail.com>
2024-04-18 12:09:14 -04:00
Kamal Chandraprakash a3dcbd4e28
KAFKA-16073: Increment the local-log-start-offset before deleting segments in memory table (#15631)
Co-authored-by: hzh0425 <642256541@qq.com>

Reviewers: Luke Chen <showuon@gmail.com>, Jun Rao <junrao@gmail.com>
2024-04-17 09:41:29 -07: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
Michael Westerby 44196984bd
KAFKA-16363 Storage tool crashes if dir is unavailable (#15733)
Reviewers: Igor Soarez <soarez@apple.com>, Federico Valeri <fedevaleri@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-17 19:03:51 +08:00
Gaurav Narula 4954c578d4
KAFKA-16559 allow defining number of disks per broker in TestKitNodes (#15730)
Reviewers: Igor Soarez <soarez@apple.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-17 13:45:37 +08:00
lixinyang 6ed3bae2c1
MINOR: Fix a few typos in clients and core tests (#15725)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Luke Chen <showuon@gmail.com>, Igor Soarez<soarez@apple.com>
Co-authored-by: lixinyang <nickxyli@tencent.com>
2024-04-16 17:50:55 +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
David Arthur e02ffd852f
KAFKA-16463 Delete metadata log on ZK broker startup (#15648)
This patch changes the behavior of the migrating ZK broker to always delete the local metadata log
during startup. This deletion is done immediately before creating the RaftManager which will
re-create the log directory and let the broker re-replicate the log from the active controller.

This new behavior is only present for ZK brokers that having migrations enabled. KRaft brokers,
even those with migrations enabled, will not delete their local metadata log. KRaft controllers are
not impacted by this change.

The rationale for this change is to make it easier for operators to re-attempt a ZK to KRaft
migration after having reverted back to ZK mode. If an operator has reverted back to ZK mode, there
will be an invalid metadata log on the disk of each broker. In order to re-attempt the migration in
the future, this log needs to be deleted. This can be pretty burdensome to the operator for large
clusters, especially since the log deletion must be done while the broker is offline.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, Igor Soarez <soarez@apple.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-12 10:21:30 -07: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
Igor Soarez 15c4ade06a
MINOR: Improve logging in AssignmentsManager (#15522)
At the moment it can be a bit difficult to troubleshoot issues related to the AssignmentsManager. Mainly because:

    Topic partitions are logged with topic ID and partition index but without the topic name.
    Directory IDs are logged without the directory path.
    Assignment reasons aren't tracked.

This patch addresses the three issues.

Reviewers: Luke Chen <showuon@gmail.com>
2024-04-12 14:13:40 +08:00
Sebastian Marsching a50ea8d4de
KAFKA-16473: Use correct cluster ID when formatting log dir. (#15658)
This fixes an issue that when starting a Docker container for the first time, the cluster ID used when formatting the log dir would not be $CLUSTER_ID but Some($CLUSTER_ID) (KAFKA-16473).

In order to be able to test the formatStorageCmd method which contained the bug, the method has been made package private.

Reviewers: cooper.tseng@suse.com, Vedarth Sharma <142404391+VedarthConfluent@users.noreply.github.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
2024-04-12 11:30:27 +05:30
Colin Patrick McCabe b67a3fa79d
KAFKA-16509: CurrentControllerId metric is unreliable in ZK mode (#15695)
The CurrentControllerId metric added by KIP-1001 is unreliable in ZK
mode. Sometimes when there is no active ZK-based controller, it still
shows the previous controller ID. Instead, it should show -1 in that
situation.

This PR fixes that by using the controller ID from the
KafkaController.scala, which is obtained directly from the controller
znode. It also adds a new test, ControllerIdMetricTest.scala.

Reviewers: David Arthur <mumrah@gmail.com>
2024-04-11 09:34:27 -07:00
ghostspiders 4aba807530
KAFKA-15610 Fix CoreUtils.swallow() test gaps (#15410)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-11 18:58:40 +08:00
Cheng-Kai, Zhang 72b823e9bd
KAFKA-16482 Eliminate the IDE warnings of accepting ClusterConfig in BeforeEach (LeaderElectionCommandTest and ProducerIdsIntegrationTest) (#15676)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-11 18:42:14 +08:00
Dongnuo Lyu 619f27015f
KAFKA-16294: Add group protocol migration enabling config (#15411)
This patch adds the `group.consumer.migration.policy` config which controls how consumer groups can be converted from classic group to consumer group and vice versa. The config is kept as an internal one while we develop the feature.

Reviewers: Jeff Kim <jeff.kim@confluent.io>, David Jacot <djacot@confluent.io>
2024-04-10 10:59:26 -07:00
Omnia Ibrahim e2e2f82f2b
KAFKA-15853 Move Sasl and SSL configs out of core (#15656)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-10 21:57:52 +08:00
Igor Soarez f6c9feea76
KAFKA-16297: Race condition while promoting future replica (#15557)
If a future replica doesn't get promoted, any directory reassignment sent to the controller should be reversed.

The current logic is already addressing the case when a replica hasn't yet been promoted and the controller hasn't yet acknowledged the directory reassignment. However, it doesn't cover the case where the replica does not get promoted due to a directory failure after the controller has acknowledged the reassignment but before the future replica catches up again and is promoted to main replica.

Reviewers: Luke Chen <showuon@gmail.com>
2024-04-10 17:57:05 +08:00
Phuc-Hong-Tran a4d6456872
KAFKA-15538: Client support for java regex based subscription (#15585)
Fully implemented legacy subscription using Pattern for AsyncKafkaConsumer.
Enabled related tests for subscription using Pattern in PlaintextConsumerTest.

Reviewers: Lianet Magrans <lianetmr@gmail.com>, Kirk True <ktrue@confluent.io>, David Jacot <djacot@confluent.io>, Bruno Cadonna <cadonna@apache.org>
2024-04-10 09:20:46 +02:00
Chia-Ping Tsai 9a6760f130
KAFKA-16310 ListOffsets doesn't report the offset with maxTimestamp a… (#15621)
We do iterate the records to find the offsetOfMaxTimestamp instead of returning the cached one when handling ListOffsetsRequest.MAX_TIMESTAMP, since it is hard to align all paths to get correct offsetOfMaxTimestamp. The known paths are shown below.

1. convertAndAssignOffsetsNonCompressed -> we CAN get correct offsetOfMaxTimestamp when validating all records
2. assignOffsetsNonCompressed -> ditto
3. validateMessagesAndAssignOffsetsCompressed -> ditto
4. validateMessagesAndAssignOffsetsCompressed#buildRecordsAndAssignOffsets -> ditto
5. appendAsFollow#append#analyzeAndValidateRecords -> we CAN'T get correct offsetOfMaxTimestamp as iterating all records is expensive when fetching records from leader
6. LogSegment#recover -> ditto

Reviewers: Jun Rao <junrao@gmail.com>
2024-04-10 11:36:07 +08:00
PoAn Yang 983a94a89e
KAFKA-16487 Support to define server properties by ClusterTestDefaults (#15687)
Sometimes we want to define server properties for all test cases, and using BeforeEach to modify the ClusterConfig is the only way. The side effect is that the IDE does not like the style since IDE can't recognize custom ParameterResolver of ClusterConfig.

The alternative is that we can take ClusterInstance from constructor first, and then we modify the inner ClusterConfig in BeforeEach phase. However, that may confuse users about the life cycle of "configs".

In short, I prefer to define the server property by ClusterTestDefaults. It already includes some server-side default property, and we can enhance that to deal with more existent test case.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-10 11:09:24 +08:00
Igor Soarez 3ad648243c
KAFKA-13907 Fix hanging ServerShutdownTest.testCleanShutdownWithKRaftControllerUnavailable (#12174)
Add new method shutdown(Duration) to accept timeout argument. We can leverage the new method to run non-graceful shutdown in testing.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-10 00:43:12 +08:00
Kuan-Po (Cooper) Tseng 169ed60fe1
KAFKA-16477 Detect thread leaked client-metrics-reaper in tests (#15668)
After profiling the kafka tests, tons of client-metrics-reaper thread not cleanup after BrokerServer shutdown.
The thread client-metrics-reaper comes from ClientMetricsManager#expirationTimer, and BrokerServer#shudown doesn't close ClientMetricsManager which let the thread still runs in background.

Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-09 05:07:33 +08:00
vamossagar12 22f9eee89b
KAFKA-16481: Fixing flaky test kafka.server.ReplicaManagerTest#testRemoteLogReaderMetrics (#15677)
Adding NPE check in RLM.

Reviewers: Luke Chen <showuon@gmail.com>
2024-04-08 20:15:54 +08:00
Philip Nee 4e0578fb81
KAFKA-16156: beginningOrEndOffsets does not need to build an OffsetAndTimestamps object upon completion (#15525)
A subtle difference in the behavior of the two API causes the failures with Invalid negative timestamp.

In this PR, the list offsets response will be processed differently based on the API. For beginingOffsets/endOffsets - the offset response should be directly returned.

For offsetsForTimes - A OffsetAndTimestamp object is constructed for each requested TopicPartition before being returned.

The reason beginningOffsets and endOffsets - We are expecting a -1 timestamp from the response which subsequently causes the invalid timestamp exception because the original code tries to construct an OffsetAndTimestamp object upon returning.

In this PR, the following missing tasks are added:

short-circuit both beginningOrEndOffsets
Test both API (beginningOrEndOffsets, OffsetsForTime)
Seems like we don't have tests for this API: Note it is presented in other IntegrationTests but they are added to test Async consumer

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Lianet Magrans <lianetmr@gmail.com>
2024-04-08 13:04:58 +02:00
Erik van Oosten 8e61f04228
MINOR: Fix usage of none in javadoc (#15674)
- Use `Empty` instead of 'none' when referring to `Optional` values.
- `Headers.lastHeader` returns `null` when no header is found.
- Fix minor spelling mistakes.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-08 08:43:05 +08:00
Nikolay 4b2278ff9c
MINOR: Default test name added to core (#15667)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-07 21:43:16 +08:00
Omnia Ibrahim e798bed198
KAFKA-16234: Log directory failure re-creates partitions in another logdir automatically (#15335)
This pr fixes the bug created by #15263 which caused topic partition to be recreated whenever the original log dir is offline: Log directory failure re-creates partitions in another logdir automatically

Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Igor Soarez <soarez@apple.com>, Gaurav Narula <gaurav_narula2@apple.com>, Proven Provenzano <pprovenzano@confluent.io>
2024-04-06 14:36:26 +08:00
Andras Katona 5e4c7dae22
KAFKA-15915: Flaky ProducerIdManagerTest error injection fix (#15605)
testUnrecoverableErrors was flaky as the wanted error either affected the next block request (prefecthing) or just missed that.

First I tried to wait for the background thread to be finished before setting the Errors.X. But then it consistently failed, because the generateProducerId call does prefetching too and after a successful producer id generation we set the error and expected that it will fail again with coordinator-load-in-progress exception but since the block was prefetched, it was able to serve us with a proper producer id.

    calling generateProducerId --> no current block exists, so requesting block --> CoordinatorLoadInProgressException
    asserting exception
    calling generateProducerId again --> prefetching, requesting the next block --> giving back the producer id from the first block
    asserting received producer id
    setting error -- waiting for the background callback(s) to be finished first
    calling generateProducerId, expecting CoordinatorLoadInProgressException, but --> works like 2), just the prefetching callback is failing due to the error we set before

Note: without the waiting for the background thread completions the error setting could happened before the 2) step's callback or after that, the test was written in a way that it expected to happen before the cb.

This was the point I realised that we need to have a queue to control the responses rather than trying to do it in the middle of the test method.

Errors can be passed in a queue at creation of the mock id manager instead modifying on-the-fly.
In the queue we're specifying Errors, how the background thread (which imitates the controllerChannel) should behave, return an error or a proper response and call the callback accordingly with that.

I was able to simplify the mock manager id class as well, no need for the maybeRequestNextBlock overriding if the errors are handled this way via a queue.

Reviewers: Igor Soarez <soarez@apple.com>, Daniel Urban <urb.daniel7@gmail.com>, Viktor Somogyi-Vass <viktorsomogyi@gmail.com>
2024-04-05 14:59:27 +02:00
PoAn Yang 21479a31bd
KAFKA-16413 add FileLockTest (#15624)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-03 21:13:45 +08:00
Greg Harris 3208c5f487
MINOR: AbstractConfig cleanup Part 2 (#15639)
Reviewers:  Manikumar Reddy <anikumar.reddy@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-03 20:05:16 +08:00
Kuan-Po (Cooper) Tseng cc6b919212
KAFKA-16435 Add test for KAFKA-16428 (#15635)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-02 04:21:10 +08:00
Gaurav Narula 40e87ae35b
KAFKA-15823: disconnect from controller on AuthenticationException (#14760)
This PR changes the handling of authenticationException on a request from the node to the controller.

We disconnect controller connection and invalidate the cache so that the next run of the thread will establish a connection with the (potentially) updated controller.

Reviewers: Luke Chen <showuon@gmail.com>, Igor Soarez <soarez@apple.com>, Omnia Ibrahim <o.g.h.ibrahim@gmail.com>
2024-04-01 11:55:08 +08:00
Johnny Hsu a640a81040
KAFKA-16323: fix testRemoteFetchExpiresPerSecMetric (#15463)
The variable of metrics item (kafka.server:type=DelayedRemoteFetchMetrics,name=ExpiresPerSec) is singleton object and it could be removed by other tests which are running in same JVM (and it is not recreated). Hence, verifying the metrics value is not stable to this test case.

Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
2024-04-01 10:52:53 +08:00
PoAn Yang 9a9b532d5d
KAFKA-16447: Fix failed ReplicaManagerTest (#15630)
The change in https://github.com/apache/kafka/pull/15373/files#r1544335647 updated broker port from 9093 to 9094. Some of test cases check broker endpoint with fixed string 9093. I change test cases to get endpoint by broker id, so these cases will not fail if someone change the port again in the future.

Reviewers: Luke Chen <showuon@gmail.com>
2024-03-30 19:18:41 +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
Greg Harris 71bcac3b6a
MINOR: AbstractConfig cleanup (#15597)
Signed-off-by: Greg Harris <greg.harris@aiven.io>

Reviewers: Chris Egerton <chrise@aiven.io>, Mickael Maison <mickael.maison@gmail.com>, Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>, Matthias J. Sax <matthias@confluent.io>
2024-03-28 13:27:41 -07:00
Nikolay 355873aa54
MINOR: Use CONFIG suffix in ZkConfigs (#15614)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Omnia Ibrahim <o.g.h.ibrahim@gmail.com>
Co-authored-by: n.izhikov <n.izhikov@vk.team>
2024-03-28 15:52:34 +01:00
Lianet Magrans b411ac0c70
KAFKA-16406 [2] : Split consumer commit tests (#15612)
Follow-up to #15535, splitting consumer integration tests defined in the long-running PlainTextConsumerTest. This PR extracts the tests that directly relate to committing offsets. No changes in logic.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2024-03-28 14:15:56 +01:00
Kuan-Po (Cooper) Tseng 4cb6806cb8
KAFKA-16232: kafka hangs forever in the starting process if the authorizer future is not returned (#15549)
add logs before and after future waiting, to allow admin to know we're waiting for the authorizer future.

Reviewers: Luke Chen <showuon@gmail.com>
2024-03-28 10:31:22 +08:00
Colin Patrick McCabe f40c06690b
KAFKA-16428: Fix bug where config change notification znode may not get created during migration (#15608)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Luke Chen <showuon@gmail.com>
2024-03-27 12:26:36 -07:00
Sanskar Jhajharia cf1ba099c0
MINOR: Renaming the `Abortable_Transaction` error to `Transaction_Abortable` (#15609)
This is a follow-up to this PR (https://github.com/apache/kafka/pull/15486) which introduced the new ABORTABLE_TRANSACTION error as a part of KIP-890 efforts. However on further discussion, we seem to gain consensus that the error should be rather named as TRANSACTION_ABORTABLE.

This PR aims to address the same. There are no changes in the code apart from that.

Reviewers: Justine Olshan <jolshan@confluent.io>, Igor Soarez <soarez@apple.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-03-28 03:13:32 +08: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
Dongnuo Lyu 4a33bc8bbf
KAFKA-16353: Offline protocol migration integration tests (#15492)
This patch adds integration tests for offline protocol migration.

Reviewers: David Jacot <djacot@confluent.io>
2024-03-27 01:57:24 -07:00
PoAn Yang 9326476065
KAFKA-16391: remove .lock file when FileLock#destroy (#15568)
Currently, server adds a .lock file to each log folder. The file is useless after server is down.

Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-03-27 11:13:54 +08:00
Colin Patrick McCabe 8d914b543d
KAFKA-16411: Correctly migrate default client quota entities (#15584)
KAFKA-16222 fixed a bug whereby we didn't undo the name sanitization used on client quota entity names
stored in ZooKeeper. However, it incorrectly claimed to fix the handling of default client quota
entities. It also failed to correctly re-sanitize when syncronizing the data back to ZooKeeper.

This PR fixes ZkConfigMigrationClient to do the sanitization correctly on both the read and write
paths. We do de-sanitization before invoking the visitors, since after all it does not make sense to
do the same de-sanitization step in each and every visitor.

Additionally, this PR fixes a bug causing default entities to be converted incorrectly. For example,
ClientQuotaEntity(user -> null) is stored under the /config/users/<default> znode in ZooKeeper. In
KRaft it appears as a ClientQuotaRecord with EntityData(entityType=users, entityName=null).
Prior to this PR, this was being converted to a ClientQuotaRecord with EntityData(entityType=users,
entityName=""). That represents a quota on the user whose name is the empty string (yes, we allow
users to name themselves with the empty string, sadly.)

The confusion appears to have arisen because for TOPIC and BROKER configurations, the default
ConfigResource is indeed the one named with the empty (not null) string. For example, the default
topic configuration resource is ConfigResource(name="", type=TOPIC).  However, things are different
for client quotas. Default client quota entities in KRaft (and also in AdminClient) are represented
by maps with null values. For example, the default User entity is represented by Map("user" ->
null).  In retrospect, using a map with null values was a poor choice; a Map<String,
Optional<String>> would have made more sense. However, this is the way the API currently is and we
have to convert correctly.

There was an additional level of confusion present in KAFKA-16222 where someone thought that using
the ZooKeeper placeholder string "<default>" in the AdminClient API would yield a default client
quota entity. Thise seems to have been suggested by the ConfigEntityName class that was created
recently. In fact, <default> is not part of any public API in Kafka. Accordingly, this PR also
renames ConfigEntityName.DEFAULT to ZooKeeperInternals.DEFAULT_STRING, to make it clear that the
string <default> is just a detail of the ZooKeeper implementation.  It is not used in the Kafka API
to indicate defaults. Hopefully this will avoid confusion in the future.

Finally, the PR also creates KRaftClusterTest.testDefaultClientQuotas to get extra test coverage of
setting default client quotas.

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Igor Soarez <soarez@apple.com>
2024-03-26 16:49:38 -07:00
PoAn Yang 6f8d4fe26b
KAFKA-15949: Unify metadata.version format in log and error message (#15505)
There were different words for metadata.version like metadata version or metadataVersion. Unify format as metadata.version.

Reviewers: Luke Chen <showuon@gmail.com>
2024-03-26 20:09:29 +08:00
Sean Quah ad960635a9
KAFKA-16386: Convert NETWORK_EXCEPTIONs from KIP-890 transaction verification (#15559)
KIP-890 Part 1 introduced verification of transactions with the
transaction coordinator on the `Produce` and `TxnOffsetCommit` paths.
This introduced the possibility of new errors when responding to those
requests. For backwards compatibility with older clients, a choice was
made to convert some of the new retriable errors to existing errors that
are expected and retried correctly by older clients.

`NETWORK_EXCEPTION` was forgotten about and not converted, but can occur
if, for example, the transaction coordinator is temporarily refusing
connections. Now, we convert it to:
 * `NOT_ENOUGH_REPLICAS` on the `Produce` path, just like the other
   retriable errors that can arise from transaction verification.
 * `COORDINATOR_LOAD_IN_PROGRESS` on the `TxnOffsetCommit` path. This
   error does not force coordinator lookup on clients, unlike
   `COORDINATOR_NOT_AVAILABLE`. Note that this deviates from KIP-890,
   which says that retriable errors should be converted to
   `COORDINATOR_NOT_AVAILABLE`.

Reviewers: Artem Livshits <alivshits@confluent.io>, David Jacot <djacot@confluent.io>, Justine Olshan <jolshan@confluent.io>
2024-03-25 16:08:23 -07:00
Igor Soarez f8ce7feebc
KAFKA-15950: Serialize heartbeat requests (#14903)
In between HeartbeatRequest being sent and the response being handled,
i.e. while a HeartbeatRequest is in flight, an extra request may be
immediately scheduled if propagateDirectoryFailure, setReadyToUnfence,
or beginControlledShutdown is called.

To prevent the extra request, we can avoid the extra requests by checking
whether a request is in flight, and delay the scheduling if necessary.

Some of the tests in BrokerLifecycleManagerTest are also improved to
remove race conditions and reduce flakiness.

Reviewers: Colin McCabe <colin@cmccabe.xyz>, Ron Dagostino <rdagostino@confluent.io>, Jun Rao <junrao@gmail.com>
2024-03-25 10:31:19 -07:00
Lianet Magrans 51c9b0d0ad
KAFKA-16406: Splitting consumer integration test (#15535)
Splitting consumer integration tests to allow for parallelization and reduce build times. This PR is only extracting tests from PlainTextConsumerTest into separate files, no changes in logic. Grouping tests by the feature they relate to so that they can be easily found

Reviewers: Andrew Schofield <aschofield@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>
2024-03-25 15:34:52 +01:00
Sanskar Jhajharia 2e8d69b78c
KAFKA-16314: Introducing the AbortableTransactionException (#15486)
As a part of KIP-890, we are introducing a new class of Exceptions which when encountered shall lead to Aborting the ongoing Transaction. The following PR introduces the same with client side handling and server side changes.

On client Side, the code attempts to handle the exception as an Abortable error and ensure that it doesn't take the producer to a fatal state. For each of the Transactional APIs, we have added the appropriate handling. For the produce request, we have verified that the exception transitions the state to Aborted.
On the server side, we have bumped the ProduceRequest, ProduceResponse, TxnOffestCommitRequest and TxnOffsetCommitResponse Version. The appropriate handling on the server side has been added to ensure that the new error case is sent back only for the new clients. The older clients will continue to get the old Invalid_txn_state exception to maintain backward compatibility.

Reviewers: Calvin Liu <caliu@confluent.io>, Justine Olshan <jolshan@confluent.io>
2024-03-22 11:26:07 -07:00
Alyssa Huang 03f7b5aa3a
KAFKA-16206: Fix unnecessary topic config deletion during ZK migration (#14206)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ron Dagostino <rndgstn@gmail.com>
2024-03-21 15:38:42 +01:00
Nikolay b6183a4134
KAFKA-14589 ConsumerGroupCommand rewritten in java (#14471)
This PR contains changes to rewrite ConsumerGroupCommand in java and transfer it to tools module

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-20 15:34:45 +08:00
PoAn Yang 34d365fd8a
KAFKA-16222: desanitize entity name when migrate client quotas (#15481)
The entity name is sanitized when it's in Zk mode.
We didn't desanitize it when we migrate client quotas. Add Sanitizer.desanitize to fix it.

Reviewers: Luke Chen <showuon@gmail.com>
2024-03-20 14:53:23 +08:00
Kuan-Po (Cooper) Tseng 12a1d85362
KAFKA-12187 replace assertTrue(obj instanceof X) with assertInstanceOf (#15512)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-20 10:36:25 +08:00
Johnny Hsu bf3f088c94
KAFKA-16341 fix the LogValidator for non-compressed type (#15476)
- Fix the verifying logic. If it's LOG_APPEND_TIME, we choose the offset of the first record. Else, we choose the record with the maxTimeStamp.
- rename the shallowOffsetOfMaxTimestamp to offsetOfMaxTimestamp

Reviewers: Jun Rao <junrao@gmail.com>, Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-03-19 23:00:30 +08:00
Artem Livshits 1d6e0b8727
KAFKA-16352: Txn may get get stuck in PrepareCommit or PrepareAbort state (#15524)
Now the removal of entries from the transactionsWithPendingMarkers map
checks the value and all pending marker operations keep the value along
with the operation state.  This way, the pending marker operation can
only delete the state it created and wouldn't accidentally delete the
state from a different epoch (which could lead to "stuck" transactions).

Reviewers: Justine Olshan <jolshan@confluent.io>
2024-03-18 19:08:55 -07:00
José Armando García Sancio 67cb742b54
MINOR; Log reason for deleting a kraft snapshot (#15478)
There are three reasons why KRaft would delete a snapshot. One, it is older than the retention time. Two, the total number of bytes between the log and the snapshot excess the configuration. Three, the latest snapshot is newer than the log.

This change allows KRaft to log the exact reason why a snapshot is getting deleted.

Reviewers: David Arthur <mumrah@gmail.com>, Hailey Ni <hni@confluent.io>
2024-03-18 12:06:42 -07:00
Edoardo Comar e9c50b1f4b
KAFKA-16369: Broker may not shut down when SocketServer fails to bind as Address already in use (#15530)
* KAFKA-16369: wait on enableRequestProcessingFuture

Add a Wait in in KafkaServer (ZK mode) for all the SocketServer ports
to be open, and the Acceptors to be started

The BrokerServer (KRaft mode) had such a wait,
which was missing from the KafkaServer (ZK mode).

Add unit test.
2024-03-18 10:14:43 +00:00
Kamal Chandraprakash e4c53d093e
KAFKA-15206: Fix the flaky RemoteIndexCacheTest.testClose test (#15523)
It is possible that due to resource constraint, ShutdownableThread#run might be called later than the ShutdownableThread#close method.

Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>
2024-03-15 10:33:40 +08:00
Luke Chen 834efa6606
KAFKA-16342 fix getOffsetByMaxTimestamp for compressed records (#15474)
Fix getOffsetByMaxTimestamp for compressed records.

This PR adds:

1) For inPlaceAssignment case, compute the correct offset for maxTimestamp when traversing the batch records, and set to ValidationResult in the end, instead of setting to last offset always.

2) For not inPlaceAssignment, set the offsetOfMaxTimestamp for the log create time, like non-compressed, and inPlaceAssignment cases, instead of setting to last offset always.

3) Add tests to verify the fix.

Reviewers: Jun Rao <junrao@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2024-03-15 06:09:45 +08:00
David Mao 37212bb242
MINOR: AddPartitionsToTxnManager performance optimizations (#15454)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Justine Olshan <jolshan@confluent.io>
2024-03-14 18:53:26 +01:00
José Armando García Sancio 722967a2b7
MINOR; Make string from array (#15526)
If toString is called on an array it returns the string representing the object reference.  Use mkString instead to print the content of the array.

Reviewers: Luke Chen <showuon@gmail.com>, Justine Olshan <jolshan@confluent.io>, Lingnan Liu <liliu@confluent.io>
2024-03-13 14:36:03 -07:00
Kamal Chandraprakash 66ed6d3b26
KAFKA-16146: Checkpoint log-start-offset for remote log enabled topics (#15201)
The log-start-offset was not getting flushed to the checkpoint file due to the check where we compare the log-start-offset with the localLog first segment base offset only. This change makes sure that tiered storage enabled topics will always try to add their entries in the log-start-offset checkpoint file.

Reviewers: Jun Rao <junrao@gmail.com>, Satish Duggana <satishd@apache.org>
2024-03-11 14:50:29 +05:30
Gaurav Narula 808bf07fb7
MINOR: Cleanup log.dirs in ReplicaManagerTest on JVM exit (#15289)
- Scala TestUtils now delegates to the function in JTestUtils
- The function is modified such that we delete the rootDir on JVM exit if it didn't exist prior to the function being invoked.

Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-03-10 03:12:39 +08:00
Johnny Hsu 3fcaa9ccc0
MINOR: remove the copy constructor of LogSegment (#15488)
In the LogSegment, the copy constructor is only used in LogLoaderTest

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-10 03:06:41 +08:00
Nikolay 414365979e
KAFKA-14589 [4/4] Tests of ConsoleGroupCommand rewritten in java (#15465)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-09 03:54:39 +08:00
Nikolay 5f4806fd1c
KAFKA-14589 [2/4] Tests of ConsoleGroupCommand rewritten in java (#15363)
This PR is part of #14471
It 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-07 07:44:17 +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
John Yu 554fa57af8
KAFKA-16209 : fetchSnapshot might return null if topic is created before v2.8 (#15444)
Change the function with a better way to deal with the NULL pointer exception.

Reviewers: Luke Chen <showuon@gmail.com>
2024-03-06 09:00:58 +08:00
Gyeongwon, Do 1ca9391285
MINOR: Remove controlPlaneRequestProcessor in BrokerServer (#15245)
It seems likely that BrokerServer was built upon the KafkaServer codebase.(#10113)
KafkaServer, using Zookeeper, separates controlPlane and dataPlane to implement KIP-291.
In KRaft, the roles of DataPlane and ControlPlane in KafkaServer seem to be divided into BrokerServer and ControllerServer.

It appears that the initial implementation of BrokerServer initialized and used the controlPlaneRequestProcessor, but it seems to have been removed, except for the code used in the shutdown method, through subsequent modifications.(#10931)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-06 05:16:35 +08:00
Colin Patrick McCabe 2c1943d836
MINOR: remove test constructor for PartitionAssignment (#15435)
Remove the test constructor for PartitionAssignment and remove the TODO.
Also add KRaftClusterTest.testCreatePartitions to get more coverage for
createPartitions.

Reviewers: David Arthur <mumrah@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-03-05 12:02:19 -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
Christo Lolov 55a6d30ccb
KAFKA-16154: Broker returns offset for LATEST_TIERED_TIMESTAMP (#15213)
This is the first part of the implementation of KIP-1005

The purpose of this pull request is for the broker to start returning the correct offset when it receives a -5 as a timestamp in a ListOffsets API request

Reviewers: Luke Chen <showuon@gmail.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Satish Duggana <satishd@apache.org>
2024-02-29 08:19:55 +08:00
Lianet Magrans 9bc9fae942
KAFKA-16258: callback to release assignment when stale member leaves group (#15415)
Introduce call to onPartitionsLost callback to release assignment when a consumer pro-actively leaves the group due to poll timer expired.

When the poll timer expires, the member sends a leave group request (reusing same existing LEAVING state and logic), and then transitions to STALE to release it assignment and wait for the poll timer reset. Once both conditions are met, the consumer transitions out of the STALE state to rejoin the group. Note that while on this STALE state, the member is not part of the group so it does not send heartbeats.

This PR also includes the fix to ensure that while STALE or in any other state where the member is not in the group, heartbeat responses that may be received are ignored.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2024-02-26 11:39:33 +01:00
Yang Yu b4e96913cc
KAFKA-16265: KIP-994 (Part 1) Minor Enhancements to ListTransactionsRequest (#15384)
Introduces a new filter in ListTransactionsRequest API. This enables caller to filter on transactions that have been running for longer than a certain duration of time.

This PR includes the following changes:

bumps version for ListTransactionsRequest API to 1. Set the durationFilter to -1L when communicating with an older broker that does not support version 1.
bumps version for ListTransactionsResponse to 1 without changing the response structure.
adds durationFilter option to kafka-transactions.sh --list
Tests:

Client side test to build request with correct combination of duration filter and API version: testBuildRequestWithDurationFilter
Server side test to filter transactions based on duration: testListTransactionsFiltering
Added test case for kafka-transactions.sh change in TransactionsCommandTest

Reviewers: Justine Olshan <jolshan@confluent.io>, Raman Verma <rverma@confluent.io>
2024-02-24 06:09:23 -08:00
Lucas Brutschy 7ac50a8611
KAFKA-16152: Fix PlaintextConsumerTest.testStaticConsumerDetectsNewPartitionCreatedAfterRestart (#15419)
The group coordinator expects the instance ID to always be sent when
leaving the group in a static membership configuration, see

ea94507679/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java (L814)

The failure was silent, because the group coordinator does not log
failed requests and the consumer doesn't wait for the heartbeat response
during close.

Reviewers: Matthias J. Sax <matthias@confluent.io>, Kirk True <ktrue@confluent.io>, Bruno Cadonna <cadonna@apache.org>
2024-02-23 16:43:50 +01:00
Omnia Ibrahim ead2431c37
MINOR: Remove unwanted debug line in LogDirFailureTest (#15371)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Justine Olshan <jolshan@confluent.io>, Igor Soarez <soarez@apple.com>
2024-02-20 11:25:17 +01:00
Lucas Brutschy 5854139cd8
KAFKA-16243: Make sure that we do not exceed max poll interval inside poll (#15372)
The consumer keeps a poll timer, which is used to ensure liveness of the application thread. The poll timer automatically updates while the Consumer.poll(Duration) method is blocked, while the newer consumer only updates the poll timer when a new call to Consumer.poll(Duration) is issued. This means that the kafka-console-consumer.sh tools, which uses a very long timeout by default, works differently with the new consumer, with the consumer proactively rejoining the group during long poll timeouts.

This change solves the problem by (a) repeatedly sending PollApplicationEvents to the background thread, not just on the first call of poll and (b) making sure that the application thread doesn't block for so long that it runs out of max.poll.interval.

An integration test is added to make sure that we do not rejoin the group when a long poll timeout is used with a low max.poll.interval.

Reviewers: Lianet Magrans <lianetmr@gmail.com>, Andrew Schofield <aschofield@confluent.io>, Bruno Cadonna <cadonna@apache.org>
2024-02-20 10:48:36 +01:00
runom a26a1d847f
MINOR: fix MetricsTest.testBrokerTopicMetricsBytesInOut (#14744)
The assertion to check BytesOut doesn't include replication was performed before replication occurred.
This PR fixed the position of the assertion.

Reviewers: Luke Chen <showuon@gmail.com>
2024-02-20 11:23:06 +08:00
Josep Prat b71999be95
MINOR: Clean up core modules (#15279)
This PR cleans up: metrics, migration, network, raft, security, serializer, tools, utils, and zookeeper package classes

Mark methods and fields private where possible
Annotate public methods and fields
Remove unused classes and methods
Make sure Arrays are not printed with .toString
Optimize minor warnings

Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-02-19 16:54:50 +01:00
Lucas Brutschy 1442862bbd
KAFKA-16009: Fix PlaintextConsumerTest. testMaxPollIntervalMsDelayInRevocation (#15383)
The wake-up mechanism in the new consumer is preventing from committing within a rebalance listener callback. The reason is that we are trying to register two wake-uppable actions at the same time.

The fix is to register the wake-uppable action more closely to where we are in fact blocking on it, so that the action is not registered when we execute rebalance listeneners and callback listeners.

Reviewers: Bruno Cadonna <cadonna@apache.org>
2024-02-19 15:33:37 +01:00
Kirk True 051d4274da
KAFKA-16167: re-enable PlaintextConsumerTest.testAutoCommitOnCloseAfterWakeup (#15358)
This integration test is now passing, presumably based on recent related changes. Re-enabling to ensure it is included in the test suite to catch any regressions.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2024-02-16 09:05:02 +01:00
Lucas Brutschy e8c70fce26
KAFKA-16155: Re-enable testAutoCommitIntercept (#15334)
The main bug causing this test to fail as described in the ticket was already fixed.

The test is still flaky if unchanged, because in the new consumer, the assignment can
change in between two polls. Interceptors are only executed inside poll (and have to be,
since they must run as part of the application thread), so we need to modify the
integration test to call poll once after observing that the assignment changed.

Reviewers: Bruno Cadonna <bruno@confluent.io>
2024-02-14 16:09:48 +01:00
Omnia Ibrahim be6653c8bc
KAFKA-16225 [1/N]: Set metadata.log.dir to broker in KRAFT mode in integration test
Fix the flakiness of LogDirFailureTest by setting a separate metadata.log.dir for brokers in KRAFT mode.

The test was flaky because as we call causeLogDirFailure some times we impact the first log.dir which also is KafkaConfig.metadataLogDir as we don't have metadata.log.dir. So to fix the flakiness we need to explicitly set metadata.log.dir to diff log dir than the ones we could potentially fail for the tests. 

This is part 1 of the fixes. Delivering them separately as the other issues were not as clear cut.

Reviewers: Gaurav Narula <gaurav_narula2@apple.com>, Justine Olshan <jolshan@confluent.io>, Greg Harris <greg.harris@aiven.io>
2024-02-13 14:13:53 -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
Gantigmaa Selenge fed3c3da84
KAFKA-14822: Allow restricting File and Directory ConfigProviders to specific paths (#14995)
Reviewers: Greg Harris <gharris1727@gmail.com>, Mickael Maison <mickael.maison@gmail.com>
2024-02-13 18:28:28 +01:00
Divij Vaidya 011d238268
MINOR: Fix package name for FetchFromFollowerIntegrationTest (#15353)
Reviewers: Omnia Ibrahim <o.g.h.ibrahim@gmail.com>, Josep Prat <josep.prat@aiven.io>
2024-02-13 12:10:49 +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
Gyeongwon, Do 489a7dd71e
MINOR: Improve Code Style (#15319)
- Removing ! and Unused Imports
- Put a space after the control structure's defining keyword.
- remove unnecessary whitespace a space after the method name in higher-order function invocations.

Reviewers: Divij Vaidya <diviv@amazon.com>
2024-02-09 12:07:20 +01:00
David Arthur 116bc000c8
MINOR: fix scala compile issue (#15343)
Reviewers: David Jacot <djacot@confluent.io>
2024-02-08 15:44:42 -08:00
David Arthur c000b1fae2
MINOR: Fix some MetadataDelta handling issues during ZK migration (#15327)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-02-07 12:54:59 -08:00
Gyeongwon, Do a63131aab8
KAFKA-15717: Added KRaft support in LeaderEpochIntegrationTest (#15225)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-02-05 16:57:10 +01:00
Ritika Reddy 68745ef21a
KAFKA-15460: Add group type filter to List Groups API (#15152)
This patch adds the support for filtering groups by types (Classic or Consumer) to both the old and the new group coordinators.

Reviewers: David Jacot <djacot@confluent.io>
2024-02-05 00:56:39 -08:00
Gaurav Narula 3db14ec62a
KAFKA-16157: fix topic recreation handling with offline disks (#15263)
In Kraft mode, the broker fails to handle topic recreation correctly with broken disks. This is because ReplicaManager tracks HostedPartitions which are on an offline disk but it doesn't associate TopicId information with them.

This change updates HostedPartition.Offline to associate topic id information. We also update the log creation logic in Partition::createLogInAssignedDirectoryId to not just rely on targetLogDirectoryId == DirectoryId.UNASSIGNED to determine if the log to be created is "new".

Please refer to the comments in https://issues.apache.org/jira/browse/KAFKA-16157 for more information.

Reviewers: Luke Chen <showuon@gmail.com>, Omnia Ibrahim <o.g.h.ibrahim@gmail.com>, Gaurav Narula <gaurav_narula2@apple.com>
2024-02-03 14:40:40 +08:00
Colin Patrick McCabe 4169ac9f5d
KAFKA-16180: Fix UMR and LAIR handling during ZK migration (#15293)
While migrating from ZK mode to KRaft mode, the broker passes through a "hybrid" phase, in which it
receives LeaderAndIsrRequest and UpdateMetadataRequest RPCs from the KRaft controller. For the most
part, these RPCs can be handled just like their traditional equivalents from a ZK-based controller.
However, there is one thing that is different: the way topic deletions are handled.

In ZK mode, there is a "deleting" state which topics enter prior to being completely removed.
Partitions stay in this state until they are removed from the disks of all replicas. And partitions
associated with these deleting topics show up in the UMR and LAIR as having a leader of -2 (which
is not a valid broker ID, of course, because it's negative). When brokers receive these RPCs, they
know to remove the associated partitions from their metadata caches, and disks. When a full UMR or
ISR is sent, deleting partitions are included as well.

In hybrid mode, in contrast, there is no "deleting" state. Topic deletion happens immediately. We
can do this because we know that we have topic IDs that are never reused. This means that we can
always tell the difference between a broker that had an old version of some topic, and a broker
that has a new version that was re-created with the same name. To make this work, when handling a
full UMR or LAIR, hybrid brokers must compare the full state that was sent over the wire to their
own local state, and adjust accordingly.

Prior to this PR, the code for handling those adjustments had several major flaws. The biggest flaw
is that it did not correctly handle the "re-creation" case where a topic named FOO appears in the
RPC, but with a different ID than the broker's local FOO. Another flaw is that a problem with a
single partition would prevent handling the whole request.

In ZkMetadataCache.scala, we handle full UMR requests from KRaft controllers by rewriting the UMR
so that it contains the implied deletions. I fixed this code so that deletions always appear at the
start of the list of topic states. This is important for the re-creation case since it means that a
single request can both delete the old FOO and add a new FOO to the cache. Also, rather than
modifying the requesst in-place, as the previous code did, I build a whole new request with the
desired list of topic states. This is much safer because it avoids unforseen interactions with
other parts of the code that deal with requests (like request logging). While this new copy may
sound expensive, it should actually not be. We are doing a "shallow copy" which references the
previous list topic state entries.

I also reworked ZkMetadataCache.updateMetadata so that if a partition is re-created, it does not
appear in the returned set of deleted TopicPartitions. Since this set is used only by the group
manager, this seemed appropriate. (If I was in the consumer group for the previous iteration of
FOO, I should still be in the consumer group for the new iteration.)

On the ReplicaManager.scala side, we handle full LAIR requests by treating anything which does not
appear in them as a "stray replica." (But we do not rewrite the request objects as we do with UMR.)
I moved the logic for finding stray replicas from ReplicaManager into LogManager. It makes more
sense there, since the information about what is on-disk is managed in LogManager. Also, the stray
replica detection logic for KRaft mode is there, so it makes sense to put the stray replica
detection logic for hybrid mode there as well.

Since the stray replica detection is now in LogManager, I moved the unit tests there as well.
Previously some of those tests had been in BrokerMetadataPublisherTest for historical reasons.

The main advantage of the new LAIR logic is that it takes topic ID into account. A replica can be a
stray even if the LAIR contains a topic of the given name, but a different ID. I also moved the
stray replica handling earlier in the becomeLeaderOrFollower function, so that we could correctly
handle the "delete and re-create FOO" case.

Reviewers: David Arthur <mumrah@gmail.com>
2024-02-02 15:49:10 -08:00
Gaurav Narula 3d95a69a28
KAFKA-16195: ignore metadata.log.dir failure in ZK mode (#15262)
In KRaft mode, or on ZK brokers that are migrating to KRaft, we have a local __cluster_metadata
log. This log is stored in a single log directory which is configured via metadata.log.dir. If
there is no metadata.log.dir given, it defaults to the first entry in log.dirs. In the future we
may support multiple metadata log directories, but we don't yet. For now, we must abort the
process when this log directory fails.

In ZK mode, it is not necessary to abort the process when this directory fails, since there is no
__cluster_metadata log there. This PR changes the logic so that we check for whether we're in ZK
mode and do not abort in that scenario (unless we lost the final remaining log directory. of
course.)

Reviewers: Luke Chen <showuon@gmail.com>, Colin P. McCabe <cmccabe@apache.org>, Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>, Proven Provenzano <pprovenzano@confluent.io>
2024-02-02 09:47:14 -08:00
Zihao Lin dfb903fb8d
KAFKA-15728: KRaft support in DescribeUserScramCredentialsRequestNotAuthorizedTest (#14736)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-02-02 15:20:14 +01:00
David Arthur 12ce9c7f98 KAFKA-16216: Reduce batch size for initial metadata load during ZK migration
During migration from ZK mode to KRaft mode, there is a step where the kcontrollers load all of the
data from ZK into the metadata log. Previously, we were using a batch size of 1000 for this, but
200 seems better. This PR also adds an internal configuration to control this batch size, for
testing purposes.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-02-01 15:48:52 -08:00
David Jacot 6c09cc9586
KAFKA-16189; Extend admin to support ConsumerGroupDescribe API (#15253)
This patch extends the Admin client to support describing new consumer groups with the ConsumerGroupDescribe API introduced in KIP-848. Users will continue to use the `Admin#describeConsumerGroups` API. The admin client does all the magic. Basically, the admin client always tries to describe the requested groups with the ConsumerGroupDescribe API to start with. If all the groups are there, great, the job is done. If there are groups unresolved groups due to a UNSUPPORTED_VERSION or GROUP_ID_NOT_FOUND error, the admin client tries with the DescribeGroups API. The patch also adds fields to the data structure returned by `Admin#describeConsumerGroups` as stated in the KIP.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Bruno Cadonna <bruno@confluent.io>
2024-02-01 00:30:56 -08: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
Josep Prat cfc8257479
MINOR: Clean up core server classes (#15272)
* MINOR: Clean up core server classes

Mark methods and fields private where possible
Annotate public methods and fields
Remove unused classes and methods
Make sure Arrays are not printed with .toString
Optimize minor warnings
Remove unused apply method

Signed-off-by: Josep Prat <josep.prat@aiven.io>

Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-01-31 13:52:22 +01:00
Gantigmaa Selenge cdd9c62c55
KAFKA-15711: KRaft support in LogRecoveryTest (#14693)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Zihao Lin
2024-01-31 11:34:42 +01:00
David Jacot 6dd517daac
KAFKA-14505; [6/N] Avoid recheduling callback in request thread (#15176)
This patch removes the extra hop via the request thread when the new group coordinator verifies a transaction. Prior to it, the ReplicaManager would automatically re-schedule the callback to a request thread. However, the new group coordinator does not need this as it already schedules the write into its own thread. With this patch, the decision to re-schedule on a request thread or not is left to the caller.

Reviewers: Artem Livshits <alivshits@confluent.io>, Justine Olshan <jolshan@confluent.io>
2024-01-30 23:27:11 -08:00
Apoorv Mittal 016bd682fe
KAFKA-16186: Broker metrics for client telemetry (KIP-714) (#15251)
Add the broker metrics defined in KIP-714.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Jun Rao <junrao@gmail.com>
2024-01-30 15:03:09 -08: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
Gaurav Narula 4c6f975ab3 KAFKA-16162: resend broker registration on metadata update to IBP 3.7-IV2
We update metadata update handler to resend broker registration when
metadata has been updated to >= 3.7IV2 so that the controller becomes
aware of the log directories in the broker.

We also update DirectoryId::isOnline to return true on an empty list of
log directories while the controller awaits broker registration.

Co-authored-by: Proven Provenzano <pprovenzano@confluent.io>

Reviewers: Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>, Luke Chen <showuon@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
2024-01-30 10:00:07 -08:00
Gaurav Narula 9e4a4a2821
KAFKA-16204: Create partition dir for mockLog (#15288)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Omnia Ibrahim <o.g.h.ibrahim@gmail.com>
2024-01-30 15:53:24 +01:00
Josep Prat 50940fa537
MINOR: Fixes broken build (#15290)
Because of lack of implicit conversions, boolean properties need to be
passed as Strings
This is done in other parts of the code already

Signed-off-by: Josep Prat <josep.prat@aiven.io>
2024-01-30 13:29:19 +01:00
Colin P. McCabe f7feb43af3 KAFKA-14616: Fix stray replica of recreated topics in KRaft mode
When a broker is down, and a topic is deleted, this will result in that broker seeing "stray
replicas" the next time it starts up. These replicas contain data that used to be important, but
which now needs to be deleted. Stray replica deletion is handled during the initial metadata
publishing step on the broker.

Previously, we deleted these stray replicas after starting up BOTH LogManager and ReplicaManager.
However, this wasn't quite correct. The presence of the stray replicas confused ReplicaManager.
Instead, we should delete the stray replicas BEFORE starting ReplicaManager.

This bug triggered when a topic was deleted and re-created while a broker was down, and some of the
replicas of the re-created topic landed on that broker. The impact was that the stray replicas were
deleted, but the new replicas for the next iteration of the topic never got created. This, in turn,
led to persistent under-replication until the next time the broker was restarted.

Reviewers: Luke Chen <showuon@gmail.com>, Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>, Gaurav Narula <gaurav_narula2@apple.com>
2024-01-29 22:36:09 -08:00
Gantigmaa Selenge 5ad1191466
KAFKA-15720: KRaft support in DeleteTopicTest (#14846)
Reviewers: Ziming Deng<dengziming1993@gmail.com>.
2024-01-30 11:34:15 +08:00
David Arthur 16ed7357b1 KAFKA-16171: Fix ZK migration controller race #15238
This patch causes the active KRaftMigrationDriver to reload the /migration ZK state after electing
itself as the leader in ZK. This closes a race condition where the previous active controller could
make an update to /migration after the new leader was elected. The update race was not actually a
problem regarding the data since both controllers would be syncing the same state from KRaft to ZK,
but the change to the znode causes the new controller to fail on the zk version check on
/migration.

This patch also fixes a as-yet-unseen bug where the active controllers failing to elect itself via
claimControllerLeadership would not retry.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-01-29 13:51:45 -08:00
Lucas Brutschy 1a54c25fdf
KAFKA-15942: Implement ConsumerInterceptors in the async consumer (#15000)
We need to make sure to call the consumer interceptor and test its integration.

This is adding the required call in commitSync and commitAsync. The calls in commitAsync are executed using the same mechanism as commit callbacks, to ensure that we are calling the interceptors from a single thread, as is intended in the original KIP.

The interceptors also need to be invoked on auto-commits which are executed in the commit request manager. For this purpose, we share the OffsetCommitCallbackInvoker class with the background thread (it is already accessed implicitly from the background thread through a future lambda). This is done analogous to the RebalanceListenerInvoker.

Co-authored-by: John Doe zh2725284321@gmail.com
Reviewers: Bruno Cadonna <bruno@confluent.io>, Andrew Schofield <aschofield@confluent.io>, Philip Nee <pnee@confluent.io>
2024-01-29 21:26:44 +01:00
DL1231 82920ffad0
KAFKA-16095: Update list group state type filter to include the states for the new consumer group type (#15211)
While using —list —state the current accepted values correspond to the classic group type states. This patch adds the new states introduced by KIP-848. It also make the matching on the server case insensitive.

Co-authored-by: d00791190 <dinglan6@huawei.com>

Reviewers: Ritika Reddy <rreddy@confluent.io>, David Jacot <djacot@confluent.io>
2024-01-29 07:19:05 -08:00
Luke Chen 70b8c5ae8e
KAFKA-16085: Add metric value consolidated for topics on a broker for tiered storage. (#15133)
In BrokerTopicMetrics group, we'll provide not only the metric for per topic, but also the all topic aggregated metric value. The beanName is like this:
kafka.server:type=BrokerTopicMetrics,name=RemoteCopyLagSegments
kafka.server:type=BrokerTopicMetrics,name=RemoteCopyLagSegments,topic=Leader

This PR is to add the missing all topic aggregated metric value for tiered storage, specifically for gauge type metrics.

Reviewers: Divij Vaidya <divijvaidya13@gmail.com>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Christo Lolov <lolovc@amazon.com>
2024-01-27 19:13:24 +08:00
Crispin Bernier 0d4e35514f
Minor update to KafkaApisTest (#15257)
I was using the ZERO_UUID topicId instead of the actual topicId in the testFetchResponseContainsNewLeaderOnNotLeaderOrFollower introduced in #14444, updating as the actual topicId is more correct.

Reviewers: Justine Olshan <jolshan@confluent.io>
2024-01-26 13:17:25 -08: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
Justine Olshan 5eb82010ef
KAFKA-15987: Refactor ReplicaManager code for transaction verification (#15087)
I originally did some refactors in #14774, but we decided to keep the changes minimal since the ticket was a blocker. Here are those refactors:

* Removed separate append paths so that produce, group coordinator, and other append paths all call appendRecords
* AppendRecords has been simplified
* Removed unneeded error conversions in verification code since group coordinator and produce path convert errors differently, removed test for that
* Fixed incorrect capital param name in KafkaRequestHandler
* Updated ReplicaManager test to handle produce appends separately when transactions are used.

Reviewers: David Jacot <djacot@confluent.io>, Jason Gustafson <jason@confluent.io>
2024-01-26 10:01:03 -08:00
Josep Prat 2a6e420dfb
MINOR: cleanup core modules part 1 (#15252)
* MINOR: Clean up core api, cluster, common, log, admin, controller and coordinator classes

Mark methods and fields private where possible
Annotate public methods and fields
Remove unused classes and methods

Signed-off-by: Josep Prat <josep.prat@aiven.io>

Reviewers: Luke Chen <showuon@gmail.com>
2024-01-26 16:35:10 +01:00
Drawxy 706c11e3ee
MINOR: Remove unreachable if-else block in ReplicaManager.scala (#15220)
After this #13107 PR, an if-else block became unreachable. We need remove it and make the code clean.

Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>
2024-01-26 14:46:39 +08:00
Mickael Maison 80a1bf8f56
KAFKA-16003: Always create the /config/topics ZNode even for topics without configs (#15022)
Reviewers: Luke Chen <showuon@gmail.com>
2024-01-25 15:46:24 +01:00
Mickael Maison c843912d40
KAFKA-7957: Enable testMetricsReporterUpdate (#15147)
Reviewers: Luke Chen <showuon@gmail.com>
2024-01-25 10:12:01 +01: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
Lianet Magrans 839cd1438b
KAFKA-16107: Stop fetching while onPartitionsAssign completes (#15215)
This ensures that no records are fetched, or positions initialized, while the onPartitionsAssigned callback completes in the new async consumer Application thread. This is achieved using an internal mark in the subscription state, so that the partitions are not considered fetchable or requiring initializing positions until the callback completes.

Reviewers: David Jacot <djacot@confluent.io>
2024-01-24 04:34:35 -08:00
Ismael Juma 70e0dbd795
Delete unused classes (#14797)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-01-23 22:04:44 -08:00