Commit Graph

13891 Commits

Author SHA1 Message Date
PoAn Yang d95e384146
KAFKA-17508: Adding some guard for fallback deletion logic (#17154)
In KAFKA-16424, we added a fallback logic to delete the logs, but the file has no parent. It'd be better we have some guard from it.

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

Reviewers: Luke Chen <showuon@gmail.com>
2024-09-13 19:45:30 +08:00
ShivsundarR 3a79fabacf
KAFKA-17502: Modified commitSync() and close() handling in clients (#17136)
Currently the code in ShareConsumeRequestManager works on the basis that there can only be one commitSync()/close() at a time. But there is a chance these calls timeout on the application thread, but are still sent later on the background thread. This will mean the incoming commitSync()/close() will not be processed, resulting in possible loss of acknowledgements.

To cover this case, we will now have a list of AcknowledgeRequestStates to store the commitSyncs() and a separate requestState to store the close(). This queue will be processed one by one until its empty. For close(), we are still assuming there can only be one active close() at a time.

eviewers:  Andrew Schofield <aschofield@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>
2024-09-13 16:54:05 +05:30
Xuan-Zhang Gong 02e3f7cc28
KAFKA-12601 Remove deprecated `delegation.token.master.key` (#17082)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-13 17:21:46 +08:00
Bruno Cadonna 0035ac06d3
KAFKA-17489: Do not handle failed tasks as tasks to assign (#17115)
Failed tasks discovered when removed from the state updater during assignment or revocation are added to the task registry. From there they are retrieved and handled as normal tasks. This leads to a couple of IllegalStateExceptions because it breaks some invariants that ensure that only good tasks are assigned and processed.

This commit solves this bug by distinguish failed from non-failed tasks in the task registry.

Reviewer: Lucas Brutschy <lbrutschy@confluent.io>
2024-09-13 10:41:45 +02:00
Matthias J. Sax b436499557
KAFKA-17524: Add timeout to StreamThreadTest (#17180)
Adds a 30s default timeout to StreamThreadTest tests.

Reviewers: David Arthur <mumrah@gmail.com>
2024-09-12 15:17:25 -04:00
Apoorv Mittal 300e825be7
MINOR Refactored share module classes (#17178)
Reviewers: Andrew Schofield <aschofield@confluent.io>, David Arthur <mumrah@gmail.com>
2024-09-12 13:32:15 -04:00
Ken Huang 4121a8952a
KAFKA-17427: Deprecate leaking *_DOC variables in StreamsConfig (#17171)
Reviewers: Matthias J. Sax <matthias@confluent.io>
2024-09-12 10:26:00 -07:00
TengYao Chi 9b53b525e8
KAFKA-17253: Deprecate Leaking Getter Methods in Joined Helper Class (#17164)
Reviewers: Matthias J. Sax <matthias@confluent.io>
2024-09-12 10:13:22 -07:00
Oleg Bonar 2703b64c4c
MINOR: Update zstd-jni to 1.5.6-5 (#17151)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-09-12 17:47:32 +02:00
TengYao Chi 241b6e6c6f
KAFKA-17410: Disable testPollThrowsInterruptExceptionIfInterrupted for AsyncConsumer (#17172)
Reviewers: Lianet Magrans <lmagrans@confluent.io>
2024-09-12 12:41:05 +02:00
David Arthur a1f28570af
MINOR Move scripts into committer-tools (#17162)
Moving reviewers.py and kafka-merge-pr.py into committer-tools. Also include a new find-unfinished-test.py 
script which can be used for finding hanging tests on Jenkins or Github Actions.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-11 18:22:35 -04:00
JohnHuang c62c3899aa
KAFKA-12829: Remove deprecated StreamsBuilder#addGlobalStore of old Processor API (#17059)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Matthias J. Sax <matthias@confluent.io>
2024-09-11 14:22:08 -07:00
João Pedro Fonseca Dantas 30a988a768
KAFKA-8666: Improve Documentation on usage of Materialized config object (#17145)
Reviewers: Matthias J. Sax <matthias@confluent.io>
2024-09-11 14:15:34 -07:00
Kuan-Po Tseng e0ee73e98b
KAFKA-16332 Remove Deprecated builder methods for Time/Session/Join/SlidingWindows (#17126)
Removed deprecated methods:
 - TimeWindows#of
 - TimeWindows#grace
 - SessionWindows#with
 - SessionWindows#grace
 - SlidingWindows#withTimeDifferencAndGrace

Reviewers: Matthias J. Sax <matthias@confluent.io>
2024-09-11 14:12:56 -07:00
David Arthur ad19d29930
KAFKA-15073 Close stale PRs [2/n] (#17166)
As a follow-up of #13827, this patch updates the stale PR workflow to automatically close PRs that have not had activity in 120 days

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Josep Prat <josep.prat@aiven.io>
2024-09-11 15:10:03 -04:00
Abhinav Dixit f466e86bb5
KAFKA-17400: Added share fetch purgatory for delaying share fetch requests (#16969)
Introduced a share fetch purgatory on the broker which delays share fetch requests that cannot be completed instantaneously. Introduced 2 new classes -

DelayedShareFetch - Contains logic to instantaneously complete or force complete a share fetch request on timeout.
DelayedShareFetchKey - Contains the key which can be used to watch the entries within the share fetch purgatory.
ShareFetchUtils - This utility class contains functionalities required for post-processing once the replica manager fetch is completed.
There are many scenarios which can cause a share fetch request to be delayed and multiple scenarios when a delayed share fetch can be attempted to be completed. In this PR, we are only targeting the case when record lock partition limit is reached, the ShareFetch should wait for up to MaxWaitMs for records to be released.

Reviewers: David Arthur <mumrah@gmail.com>, Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>, Jun Rao <junrao@gmail.com>
2024-09-11 11:47:33 -07:00
PoAn Yang 3f4c25fe1d
KAFKA-17448: New consumer seek should update positions in background thread (#17075)
Reviewers: Lianet Magrans <lmagrans@confluent.io>, Kirk True <ktrue@confluent.io>
2024-09-11 20:08:33 +02:00
Andrew Schofield 0c4ffc682c
KAFKA-17231 Add missing node latency metrics (#17137)
This is the equivalent of #16755 for the share group consumer.

The node request-latency-max and request-latency-avg were not being recorded and thus reported as NaN for the share group consumer.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-09-12 01:56:41 +08:00
Chung, Ming-Yen 60707a5fe9
KAFKA-17392 Remove whitelist option in ConsoleConsumerOptions (#17138)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-12 01:46:16 +08:00
TaiJuWu 74f0c9dabc
MINOR: a bit refacotr for TopicCommandTest (#17161)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-12 01:17:08 +08:00
Andrew Schofield 7b865f2b22
KAFKA-17347 Add missing client-metrics option to kafka-configs.sh (#17046)
When KIP-714 was developed, the entity type of client-metrics was added to the kafka-configs.sh tool. The idea was to have two forms of specifying the name and type of a client metrics config resource, either --entity-type client-metrics --entity-name NAME or --client-metrics NAME. This style of alias is used for all of the entity types. Unfortunately, the --client-metrics form was not implemented. This PR corrects that and adds more tests.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, DL1231 <53332773+DL1231@users.noreply.github.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-09-12 01:01:03 +08:00
David Arthur 0e30209f01
KAFKA-17506 KRaftMigrationDriver initialization race (#17147)
There is a race condition between KRaftMigrationDriver running its first poll() and being notified by Raft about a leader change. If onControllerChange is called before RecoverMigrationStateFromZKEvent is run, we will end up getting stuck in the INACTIVE state.

This patch fixes the race by enqueuing a RecoverMigrationStateFromZKEvent from onControllerChange if the driver has not yet initialized. If another RecoverMigrationStateFromZKEvent was already enqueued, the second one to run will just be ignored.

Reviewers: Luke Chen <showuon@gmail.com>
2024-09-11 10:41:49 -04:00
Alieh Saeedi d04f534892
KAFKA-17109: implement exponential backoff for state directory lock (#17116)
This PR implements exponential backoff for state directory lock to increase the time between two consecutive attempts of acquiring the lock.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2024-09-11 13:26:19 +02:00
Vikas Singh b4e1deb43a MINOR: Few cleanups
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2024-09-11 15:12:53 +05:30
Chirag Wadhwa e380b7f5d1
KAFKA-17290: Added Integration tests for ShareFetch and ShareAcknowledge APIs (#16916)
This PR contains Integration Tests for Share Fetch and Share Acknowledge APIs

Reviewers:  Apoorv Mittal <apoorvmittal10@gmail.com>,  Manikumar Reddy <manikumar.reddy@gmail.com>
2024-09-11 14:26:45 +05:30
Xuan-Zhang Gong 9242723e4c
KAFKA-17435 remove use.incremental.alter.configs (#17027)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-11 14:34:05 +08:00
David Arthur 11d8069fcd
MINOR: Handle new Gradle exit code behavior in deflake.yml (#17158)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-11 14:22:54 +08:00
David Arthur a981c33bdf
MINOR Disable testMigrateTopicDeletions (#17153)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-09-10 21:56:24 -04:00
Ken Huang 51d296c86f
MINOR: Improve JavaDocs for deprecates ForeachProcessor (#17156)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Matthias J. Sax <matthias@confluent.io>
2024-09-10 18:47:55 -07:00
João Pedro Fonseca Dantas 794e9a4a52
KAFKA-14995: Automate asf.yaml collaborators refresh (#17124)
Add a Python script that analyzes our Git history to find top contributors. This can be used by committers to update
the list of contributors in .asf.yaml without a lot of tedious effort. 

Co-authored-by: stevenbooke <steviebeee55@gmail.com>
Co-authored-by: Joao Pedro Fonseca <fonsdant@gmail.com>
Reviewers: David Arthur <mumrah@gmail.com>
2024-09-10 21:29:51 -04:00
Matthias J. Sax 6fd973b4a5
KAFKA-16331: Remove EOSv1 from Kafka Streams system tests (#17108)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Bill Bejeck <bill@confluent.io>
2024-09-10 17:55:03 -07:00
xijiu 0af75c0e41
KAFKA-17458 Add 3.8 to transactions_upgrade_test.py, transactions_mixed_versions_test.py, and kraft_upgrade_test.py (#17084)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-11 02:12:25 +08:00
David Arthur 2ebe4551c3
MINOR CI Complete workflow should set statuses on apache/kafka only (#17155)
Instead of attempting to set statuses on the head repo of a PR, the CI Complete workflow should create statuses on the commit in apache/kafka. 

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-10 13:28:06 -04:00
Ken Huang e311716beb
KAFKA-17492 skip features with minVersion of 0 instead of replacing 0 with 1 when BrokerRegistrationRequest < 4 (#17128)
The 3.8 controller assumes the unknown features have min version = 0, but KAFKA-17011 replace the min=0 by min=1 when BrokerRegistrationRequest < 4. Hence, to support upgrading from 3.8.0 to 3.9, this PR changes the implementation of ApiVersionsResponse (<4) and BrokerRegistrationRequest (<4) to skip features with supported minVersion of 0 instead of replacing 0 with 1

Reviewers: Jun Rao <junrao@gmail.com>, Colin P. McCabe <cmccabe@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2024-09-11 01:16:59 +08:00
Kuan-Po Tseng 05955bc1fc
KAFKA-17224 Make ForeachProcessor internal (#17129)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-10 22:54:42 +08:00
David Jacot 31f79055ce
KAFKA-17306; Soften the validation when replaying tombstones (#16898)
This patch fixes a few buts in the replay logic of the consumer group records:
* The first issue is that the logic assumed that the group or the member exists when tombstones are replayed. Obviously, this is incorrect after a restart. The group or the member may not me there anymore if the __consumer_offsets partitions only contains tombstones for the group or the member. The patch fixes this by considering tombstones as no-ops if the entity does not exist.
* The second issue is that the logic assumed that consumer group records are always in a specific order in the log so the logic was only accepting to create a consumer group when `ConsumerGroupMemberMetadata` record is replayed. This is obviously incorrect too. During the life time of a consumer group, the records may be in different order. The patch fixes this by allowing the creating of a consumer group by any record.
* The third issue is that it is possible to replay offset commit records for a specific consumer group before the consumer group is actually created while replying its records. By default the OffsetMetadataManager creates a simple classic group to hold those offset commits. When the consumer offset records are finally replayed, the logic will fail because a classic group already exists. The patch fixes this by converting a simple classic group when records for a consumer group are replayed.

All those combinations are hard to test with unit tests. This patch adds an integration tests which reproduces some of those interleaving of records. I used them to reproduce the issues describe above.

Reviewers: TengYao Chi <kitingiao@gmail.com>, Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-09-10 07:28:36 -07:00
Frederik Rouleau cd7670d884
KAFKA-17478 Fix NPE with bad metric.reporters config (#17086)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-10 22:22:06 +08:00
David Arthur 975bc831d1
MINOR Fix trunk build scan (#17152)
Trunk builds are run off of "push" events rather than "pull_request". We were missing some logic in the is-public-fork condition that mistakenly caused some trunk builds to skip the build scan.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-10 10:09:44 -04:00
Chung, Ming-Yen e0bf580185
KAFKA-17311 Add groupProtocol parameter to testClientInstanceId, testClientInstanceIdInvalidTimeout, and testClientInstanceIdNoTelemetryReporterRegistered (#17135)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-10 22:09:32 +08:00
xijiu ac4784ec0c
KAFKA-17418: Improve markdown formatting in junit.py (#17071)
Newline characters in the failure message of tests were causing the Markdown tables to be malformed.
This patch fixes that by replacing newlines with "<br>" tags and escaping other HTML that may appear in message.

Reviewers: David Arthur <mumrah@gmail.com>
2024-09-10 09:25:03 -04:00
David Arthur 5cf3098366
MINOR Skip ci-complete unless run was success or failure (#17148)
Tighten the condition to only run the ci-complete workflow if the triggering run was success or failure. Also, 
add a status check failure if the PR did not produce the expected build scans.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-10 08:20:18 -04:00
David Arthur f41fced474
MINOR revert test logging change (#17144)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-10 08:16:29 -04:00
Dmitry Werner af8d058d8e
KAFKA-17491: Move BrokerServerMetrics to server module (#17114)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-09-10 11:14:43 +02:00
Mickael Maison beacf488d1
MINOR: Add equals/hashcode to DeprecatedRequestRateKey (#17134)
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-09-10 16:43:10 +08:00
Kuan-Po Tseng 089cbefac9
KAFKA-16830 Remove the scala version formatters support (#17127)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-10 16:04:58 +08:00
DL1231 f629b14890
KAFKA-14772; Add ConsumerGroupHeartbeat and ConsumerGroupDescribe API to AuthorizerIntegrationTest (#17044)
The patch adds ConsumerGroupHeartbeat and ConsumerGroupDescribe API to AuthorizerIntegrationTest.

Reviewers: David Jacot <djacot@confluent.io>
2024-09-10 00:17:31 -07:00
TengYao Chi d55d0e1a8f
KAFKA-17349 Make sure all IT have "quorum" input arguments (#16896)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-10 13:01:16 +08:00
TengYao Chi 2dc3ee0557 KAFKA-17497 Add e2e for zk migration with old controller (#17131)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-10 12:15:32 +08:00
David Arthur b22e848dde
MINOR: explicitly grant "statuses: write" permission in ci-complete.yml (#17142)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-09 20:50:58 -04:00
Sushant Mahajan 821c10157d
KAFKA-17367: Introduce share coordinator [2/N] (#17011)
Introduces the share coordinator. This coordinator is built on the new coordinator runtime framework. It 
is responsible for persistence of share-group state in a new internal topic named "__share_group_state".
The responsibility for being a share coordinator is distributed across the brokers in a cluster. 

Reviewers: David Arthur <mumrah@gmail.com>, Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>
2024-09-09 20:01:24 -04:00