Commit Graph

13792 Commits

Author SHA1 Message Date
Colin P. McCabe d22d58da46 rework the reconfiguration logic a bit to fix the non-reconfigurable log config 2024-09-25 15:35:03 -07:00
Colin P. McCabe 0a2752a52b Do not allow log configurations to be changed if they're not reconfigurable
- Add testLogRetentionTimeMinutesIsNotDynamicallyReconfigurable

- clean up some cases where we were using zkconnect but did not need to
2024-09-25 10:56:01 -07:00
Colin P. McCabe 297e961c3a remove testDynamicLogReconfigurableConfigsIncludesDeprecatedSynonyms 2024-09-24 13:30:17 -07:00
Colin P. McCabe 4ce9b0d664 Go back to the old behavior where synonyms cannot be used for dynamic configs 2024-09-24 10:49:39 -07:00
Colin P. McCabe 8cf31e0fcf KAFKA-17584: Fix incorrect synonym handling for dynamic log configurations
Several Kafka log configurations in have synonyms. For example, log retention can be configured
either by log.retention.ms, or by log.retention.minutes, or by log.retention.hours. There is also
a faculty in Kafka to dynamically change broker configurations without restarting the broker. These
dynamically set configurations are stored in the metadata log and override what is in the broker
properties file.

Unfortunately, these two features interacted poorly; there was a bug where the dynamic log
configuration update code ignored synonyms. For example, if you set log.retention.minutes and then
reconfigured something unrelated that triggered the LogConfig update path, the retention value that
you had configured was overwritten.

The reason for this was incorrect handling of synonyms. The code tried to treat the Kafka broker
configuration as a bag of key/value entities rather than extracting the correct retention time (or
other setting with overrides) from the KafkaConfig object.

Separately from the above bug, the code did not honor the value of dynamically configured synonyms:
setting log.retention.minutes had no effect; only log.retention.ms was honored.
2024-09-23 16:14:52 -07:00
Sean Quah 9352faa8fc
KAFKA-17495: Factor out assignor benchmark code into utils class (#17133)
ServerSideAssignorBenchmark and TargetAssignmentBuilderBenchmark have
the same topic and member subscription setup for the most part. Factor
out the commonality so that it's easier to share new setups between both
benchmarks.

Reviewers: David Jacot <djacot@confluent.io>
2024-09-23 07:55:54 -07:00
Stig Døssing a407cc3a31
KAFKA-17575 Remove unnecessary file.deleteOnExit call (#17226)
TestUtils.tempDirectory already registers a shutdown hook for deleting the temp directory. There's no reason to also call File.deleteOnExit, since that just registers another hook to do the same thing.

Reviewers: TengYao Chi <kitingiao@gmail.com>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-09-23 20:28:08 +08:00
Matthias J. Sax d063443825
MINOR: fixing JavaDocs and other cleanup (#17207)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-23 11:46:40 +08:00
Matthias J. Sax 9685aa7547
KAFKA-16331 remove EOSv1 from StreamsProducer (#17212)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-23 11:34:39 +08:00
Chia-Chuan Yu 8c2a0619e2
KAFKA-17528 Remove whitelist/blacklist from JMXReporter (#17179)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-23 11:17:01 +08:00
Matthias J. Sax bf450ebe5a
MINOR: fix generics in streams-test-utils package (#17206)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-21 22:48:38 +08:00
Matthias J. Sax 2489cf586f
KAFKA-16331: remove EOSv1 config from StreamsConfig (#17170)
Reviewers: Bill Bejeck <bill@confluent.io>
2024-09-20 15:55:03 -07:00
bboyleonp666 9a11898c8c
KAFKA-17567 Remove TestTruncate (#17234)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-21 05:43:24 +08:00
Ken Huang 79753594ca
KAFKA-16813 Add global timeout (60s) for `@ClusterTemplate`, `@ClusterTest` and `@ClusterTests` (#16957)
Reviewers: TaiJuWu <tjwu1217@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-09-21 02:36:13 +08:00
TaiJuWu 3783385dc1
KAFKA-17542: Use actions/labeler for automatic PR labeling (#17208)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Arthur <mumrah@gmail.com>
2024-09-19 19:39:12 -04:00
David Arthur 57013061c1
MINOR Fix CI workflow for push event (#17239)
On trunk, our CI runs in response to "push" events. The change in #17227 causes the workflow template to be invalid, which prevents the build from starting. This patch fixes that by defaulting to `false`

Reviewers: Justine Olshan <jolshan@confluent.io>
2024-09-19 17:07:31 -04:00
David Arthur e3983c2a44
MINOR Fix is-public-fork input type (#17227)
Fix the CI workflow to treat the `is-public-fork` input as a string. 

Also add some docs on composite actions.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-19 14:13:06 -04:00
Kuan-Po Tseng 8569cf102b
KAFKA-17356 add integration test for KAFKA-17310 (#17211)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-20 01:14:58 +08:00
David Arthur ef567bcc3f
MINOR: Group the junit parser console logs (#17229)
Use ::group:: feature of GitHub Actions to hide some of the verbose output from Parse JUnit Tests step.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-20 00:50:07 +08:00
xijiu e90b246002
KAFKA-17555 uncomment all checks of testCommonNameLoggingTrustManagerMixValidAndInvalidCertificates (#17220)
Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-09-20 00:00:36 +08:00
Jason Taylor 14c45bed5a
KAFKA-17579: Dynamic LogCleaner configurations are picked up on restart
Reviewers: Christo Lolov <lolovc@amazon.com>
2024-09-19 15:52:58 +01:00
David Arthur 31d395163e
KAFKA-17553 Fix shutdown race condition in StreamThreadTest (#17191)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-19 15:17:25 +08:00
陳昱霖(Yu-Lin Chen) 8f5cf9968f
KAFKA-17515 Fix flaky RestoreIntegrationTest.shouldInvokeUserDefinedGlobalStateRestoreListener (#17187)
Found two issues in the flaky tests: (Put the log analysis under Jira comments.)

1) The error "java.nio.file.DirectoryNotEmptyException" occurs if the flush() of kafkaStreams.close() and purgeLocalStreamsState() are triggered in the same time. (The current timeout is 5 sec, which is too short since the CI is unstable and slow).
2) Racing issue: Task to-be restored in ks-1 are rebalanced to ks-2 before entering active restoring state. So no onRestoreSuspend() was triggered.

To solve the issues:
1) Remove the timeout in kafkaStreams.close()
2) Ensure all tasks in ks-1 are active restoring before start second KafkaStreams(ks-2)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-19 14:58:04 +08:00
Jakub Scholz 09e3c12057
KAFKA-17543: Improve and clarify the error message about generated broker IDs in migration (#17210)
This PR tries to improve the error message when broker.id is set to -1 and ZK migration is enabled. It is not
needed to disable the broker.id.generation.enable option. It is sufficient to just not use it (by not setting
the broker.id to -1).

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Luke Chen <showuon@gmail.com>
2024-09-18 11:45:25 -07:00
Colin P. McCabe 3636afb737 HOTFIX: add file that was missed from KAFKA-17048 commit 2024-09-18 11:33:14 -07:00
José Armando García Sancio aee44efd40
KAFKA-17048; Update docs for KIP-853 (#17076)
Change the configurations under config/kraft to use controller.quorum.bootstrap.servers instead of controller.quorum.voters. Add comments explaining how to use the older static quorum configuration where appropriate.

In docs/ops.html, remove the reference to "tentative timelines for ZooKeeper removal" and "Tiered storage is considered as an early access feature" since they are no longer up-to-date. Add KIP-853 information.

In docs/quickstart.html, move the ZK instructions to be after the KRaft instructions. Update the KRaft instructions to use KIP-853.

In docs/security.html, add an explanation of --bootstrap-controller and document controller.quorum.bootstrap.servers instead of controller.quorum.voters.

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Alyssa Huang <ahuang@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
2024-09-18 11:23:58 -07:00
Kamal Chandraprakash 77e952687b
KAFKA-17559: Fix the flaky RemoteLogOffsetReader#testTaskQueueFullAndCancelTask (#17214)
Reviewers: David Arthur <mumrah@gmail.com>, Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>
2024-09-18 21:49:47 +05:30
David Jacot 74bebf6e3d
MINOR: Log pending join members (#17219)
I am still chasing KAFKA-17493. I was able to narrow it down to an issue with the pending join members. This patch logs them in order to help me troubleshooting it further. I will revert this change when the issue is root caused.

Reviewers: David Arthur <mumrah@gmail.com>
2024-09-18 00:34:06 -07:00
Ken Huang 95b734d3c8
KAFKA-17472 Speed Up DescribeConsumerGroupTest (#17117)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-18 04:56:45 +08:00
Colin Patrick McCabe f1e7954ad1 MINOR: update documentation link to 3.9 (#17216)
Reviewers: David Arthur <mumrah@gmail.com>
2024-09-17 07:36:48 -07:00
Matthias J. Sax aaf3fc05f8
MINOR: fix rawtype warning in StandbyTask (#17203)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-16 11:49:19 -07:00
Mickael Maison f1c011a8b5
KAFKA-14482 Move LogLoader to storage module (#17042)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-17 00:37:49 +08:00
Matthias J. Sax 21e67b3d21
KAFKA-16331 Remove EOSv1 from Kafka Streams integration tests (#17110)
Reviewers: Bill Bejeck <bbejeck@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-09-17 00:27:51 +08:00
Ivan Yurchenko d0f4d691b5
KAFKA-17322: Document required header versions in Protocol Guide (#16867)
Reviewers: Josep Prat <josep.prat@aiven.io>
2024-09-16 14:19:47 +02:00
Ivan Yurchenko ab1eb8d561
KAFKA-17494: Document message structure in Protocol Guide (#17122)
Reviewers: Josep Prat <josep.prat@aiven.io>
2024-09-16 12:03:59 +02:00
Kamal Chandraprakash 344d8a60af
KAFKA-15859 Make RemoteListOffsets call an async operation (#16602)
This is the part-2 of the KIP-1075

To find the offset for a given timestamp, ListOffsets API is used by the client. When the topic is enabled with remote storage, then we have to fetch the remote indexes such as offset-index and time-index to serve the query. Also, the ListOffsets request can contain the query for multiple topics/partitions.

The time taken to read the indexes from remote storage is non-deterministic and the query is handled by the request-handler threads. If there are multiple LIST_OFFSETS queries and most of the request-handler threads are busy in reading the data from remote storage, then the other high-priority requests such as FETCH and PRODUCE might starve and be queued. This can lead to higher latency in producing/consuming messages.

In this patch, we have introduced a delayed operation for remote list-offsets call. If the timestamp need to be searched in the remote-storage, then the request-handler threads will pass-on the request to the remote-log-reader threads. And, the request gets handled in asynchronous fashion.

Covered the patch with unit and integration tests.

Reviewers: Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-09-16 07:25:06 +08:00
Bill Bejeck e1f11c6714
MINOR: Need to split the controller bootstrap servers on ',' in list comprehenson (#17183)
Kafka Streams system tests were failing with this error:

Failed to parse host name from entry 3001@d for the configuration controller.quorum.voters.  Each entry should be in the form `{id}@{host}:{port}`.

The cause is that in kafka.py line 876, we create a delimited string from a list comprehension, but the input is a string itself, so each character gets appended vs. the bootstrap server string of host:port. To fix this, this PR adds split(',') to controller_quorum_bootstrap_servers. Note that this only applies when dynamicRaftQuorum=False

Reviewers: Alyssa Huang <ahuang@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-09-16 02:26:06 +08:00
Lianet Magrans 6744a718c2
KAFKA-17066 new consumer updateFetchPositions all in background thread (#16885)
Fix for the known issue that the logic for updating fetch positions in the new consumer was being performed partly in the app thread, party in the background thread, potentially leading to race conditions on the subscription state.

This PR moves the logic for updateFetchPositions to the background thread as a single event (instead of triggering separate events to validate, fetchOffsets, listOffsets). A new UpdateFetchPositionsEvent is triggered from the app thread and processed in the background, where it performs those same operations and updates the subscription state accordingly, without blocking the background thread.

This PR maintains the existing logic for keeping a pendingOffsetFetchRequest that does not complete within the lifetime of the updateFetchPositions attempt, and may be used on the next call to updateFetchPositions.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-09-16 01:43:45 +08:00
David Jacot f7430cf84b
MINOR: Log reason holding the completion of the join phase in the classic protocol (#17197)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-16 01:33:58 +08:00
TengYao Chi c95865437b
KAFKA-16027: Refactor testUpdatePartitionLeadership (#17083)
Reviewers: David Arthur <mumrah@gmail.com>
2024-09-15 09:38:26 -04:00
tkuramoto33 5a01945cb7
MINOR: Fix javadoc url for KafkaProducer (#17200)
Fix javadoc url for KafkaProducer

Reviewers: Luke Chen <showuon@gmail.com>
2024-09-15 20:00:29 +08:00
xijiu 7a321f29a2
KAFKA-17513 Move LogSegmentsTest to storage module (#17173)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-15 00:26:07 +08:00
David Arthur 61a8d6584f
MINOR Always publish build scan in CI Complete workflow (#17195)
This patch bring the PR and trunk builds closer in line. Rather than switching between `--scan` and `--no-scan`,
both scenarios now use `--no-scan` and rely on the CI Complete workflow to publish the scans.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-14 08:48:42 -04:00
David Jacot 70f7418660
MINOR: Increase logging verbosity in Connect integration tests (#17189)
This patch increases the verbosity of the logging in Connect's integration tests. This is to better understand the causes of the flaky tests described in KAFKA-17493.

Reviewers: Chris Egerton <chrise@aiven.io>
2024-09-14 00:04:46 -07:00
Colin Patrick McCabe d7a456ed5d
KAFKA-17543: Enforce that broker.id.generation.enable is not used when migrating to KRaft (#17192)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Arthur <mumrah@gmail.com>
2024-09-13 17:24:48 -07:00
Matthias J. Sax 6610a4d46f
KAFKA-17527: Fix NPE for null RecordContext (#17169)
Reviewers: Bruno Cadonna <bruno@confluent.io>
2024-09-13 16:34:15 -07:00
Clay Johnson 45d040d881
Update to 4.1.0 of `setup-gradle` action (#17194)
Fixes an issue with Develocity tokens expiring too soon.

Reviewers: David Arthur <mumrah@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-09-13 18:04:22 -04:00
PoAn Yang 98252cae14
KAFKA-16681 Rewrite MiniKDC by Java (#16213)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-14 04:13:40 +08:00
PoAn Yang 4692aeb671
KAFKA-17036 KIP-919 supports for createAcls, deleteAcls, describeAcls (#16493)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-14 03:40:32 +08:00
Colin Patrick McCabe d3936365bf
KAFKA-16468: verify that migrating brokers provide their inter.broker.listener (#17159)
When brokers undergoing ZK migration register with the controller, it should verify that they have
provided a way to contact them via their inter.broker.listener. Otherwise the migration will fail
later on with a more confusing error message.

Reviewers: David Arthur <mumrah@gmail.com>
2024-09-13 09:18:24 -07:00