Commit Graph

16060 Commits

Author SHA1 Message Date
Sushant Mahajan 28c53ba09a
KAFKA-19453: Ignore group not found in share group record replay. (#20076)
CI / build (push) Waiting to run Details
* When a `ShareGroup*` record is replayed in group
metadata manager, there is a call to check if the group exists. If the
group does not exist - we are throwing an exception which is
unnecessary.
* In this PR, we have added check to ignore this exception.
* New test to validate the logic has been added.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Dongnuo Lyu
<139248811+dongnuo123@users.noreply.github.com>
2025-07-02 10:10:14 +01:00
stroller 14ea11dc31
KAFKA-19371: Don't create the __remote_log_metadata topic when it already exists during broker restarts (#19899)
* The CREATE_TOPIC request gets issued only when it is clear that the
topic does not exist in the cluster.
* When the request to describe the topic gets timed-out or any exception
thrown other than UnknownTopicOrPartitionException, then the same gets
re-thrown and the describe/create topic request gets retried in the next
iteration until the initializationRetryMaxTimeoutMs gets breached.

Fixes: https://issues.apache.org/jira/browse/KAFKA-19371

Reviewers: Luke Chen <showuon@gmail.com>, Kamal Chandraprakash
<kamal.chandraprakash@gmail.com>

---------

Co-authored-by: stroller.fu <stroller.fu@zoom.us>
2025-07-02 11:22:26 +05:30
Matthias J. Sax eaa55c420b
MINOR: simplify OpenIterator (#20060)
CI / build (push) Waiting to run Details
We can remove the explicit counter for open iterators, and just use
size() on the set of open iterators we track anyway.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-07-01 12:54:49 -07:00
Lucas Brutschy 2ae85ef214
KAFKA-19429: Deflake streams_smoke_test, again (#20070)
It looks like we are checking for properties that are not guaranteed
under at_least_once, for example, exact counting (not allowing for
overcounting).

This change relaxes the validation constraint:

The TAGG topic contains effectively count-by-count results. So for
example, if we have the input without duplication

0 -> 1,2,3 we will get in TAGG 3 -> 1, since 1 key had 3 values.

with duplication:

0 -> 1,1,2,3 we will get in TAGG 4 -> 1, since 1 key had 4 values.

This makes the result difficult to compare. Since we run the smoke test
also with Exactly_Once, I propose to disable validation off TAGG under
ALOS.

Similarly, the topic AVG may overcount or undercount. The test case is
extremely similar to DIF, both performing a join and two streams, the
only difference being the mathematical operation performed, so we can
also disable this validation under ALOS with minimal loss of coverage.

Finally, the change fixes a bug that would throw a NPE when validation
of a windowed stream would fail.

Reviewers: Kirk True <kirk@kirktrue.pro>, Matthias J. Sax
 <matthias@confluent.io>
2025-07-01 21:48:07 +02:00
Sushant Mahajan 05b2601dde
KAFKA-19456: State and leader epoch should not be updated on writes. (#20079)
* If a write request with higher state than seen so far for a
specific share partition arrives at the share coordinator, the code will
create a new share snapshot and also update the internal view of the
state epoch.
* For writes with higher leader epoch, the current records are updated
with that value as well.
* The above is not the expected behavior and only initialize RPCs should
set and alter the state epoch and read RPC should set the leader epoch.
* This PR rectifies the behavior.
* Few tests have been removed.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-07-01 19:57:57 +01:00
Sushant Mahajan ac583ad2c0
KAFKA-19455: Retry persister request on metadata image issues. (#20078)
* If we get an `UNKNOWN_TOPIC_OR_PARTITION` response from the
`ShareCoordinator` is could imply a transient issue where the metadata
image is not upto date.
* In this case it makes sense to retry the request to give time for data
to be available.
* In this PR, we are making the required change.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-07-01 19:47:59 +01:00
Tsung-Han Ho (Miles Ho) ad934d3202
MINOR: Remove threadNamePrefix parameter from ReplicaManager and ReplicaFetcherManager (#20069)
CI / build (push) Waiting to run Details
- remove `threadNamePrefix` from `ReplicaManager` constructor
- update `BrokerServer` to use updated constructor
- remove `threadNamePrefix` from `ReplicaFetcherManager`

Reviewers: PoAn Yang <payang@apache.org>, TengYao Chi
 <frankvicky@apache.org>
2025-07-01 20:36:50 +08:00
Kirk True 3c902bacc0
KAFKA-19152: Add top-level documentation for OAuth flows (#20025)
CI / build (push) Waiting to run Details
Adds documentation to support the OAuth additions from KIP-768 and
KIP-1139.

The existing documentation is heavily geared toward Kafka's support for
non-production OAuth usage. Since this mode is still supported, it
should not be removed. However, with the addition of the production
OAuth usage, the documentation is less than succinct because it has a
bit of a split personality issue.
2025-07-01 12:39:37 +05:30
Matthias J. Sax c8f83592b2
MINOR: improve StreamsProducer error handling (#20058)
CI / build (push) Waiting to run Details
StreamProducer may timeout in sendOffsetsToTransaction() or
commitTransaction() call. To distinguish both cases, we should make both
calls in individual try-catch blocks.

Reviewers: Bill Bejeck<bbejeck@apache.org>
2025-06-30 15:03:35 -07:00
Jhen-Yung Hsu 64aebb5621
MINOR: remove unused FlattenedIterator (#20067)
CI / build (push) Waiting to run Details
Remove FlattenedIterator. It’s no longer used anywhere after
https://github.com/apache/kafka/pull/20037.

Reviewers: TengYao Chi <kitingiao@gmail.com>, Lan Ding
 <isDing_L@163.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-06-30 20:02:08 +08:00
Lucas Brutschy 53d654ab6e
KAFKA-19379: Basic upgrade guide for KIP-1071 EA (#20029)
CI / build (push) Waiting to run Details
Basic documentation describing:   - That it's in EA now

 - What it does

 - What features are not yet supported

 - How to enable it / disable it

 - Any changes in the interfaces

          - kafka-streams-groups.sh

          - StreamsGroupDescribe

 - How to provide feedback

Reviewers: Andrew Schofield <aschofield@confluent.io>, Matthias J. Sax
 <matthias@confluent.io>

---------

Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
Co-authored-by: Matthias J. Sax <mjsax@apache.org>
2025-06-30 09:28:22 +02:00
Sean Quah 08eda2ebed
KAFKA-19445: Fix coordinator runtime metrics sharing sensors (#20062)
When sensors are shared between different metric groups, data from all
groups is combined and added to all metrics under each sensor. This
means that different metric groups will report the same values for their
metrics.

Prefix sensor names with metric group names to isolate metric groups.

Reviewers: Yung <yungyung7654321@gmail.com>, Sushant Mahajan
<smahajan@confluent.io>, Dongnuo Lyu <dlyu@confluent.io>, TengYao Chi
<frankvicky@apache.org>
2025-06-30 15:14:39 +08:00
Yunchi Pang 975fe75d25
MINOR: Make feature lists immutable (#20052)
Replaces `.collect(Collectors.toList())` with `.toList()` for feature
collections, ensuring they are immutable and preventing accidental
modification.

Reviewers: TaiJuWu <tjwu1217@gmail.com>, Yung
<yungyung7654321@gmail.com>, Ken Huang <s7133700@gmail.com>, TengYao Chi
<frankvicky@apache.org>
2025-06-30 12:35:46 +08:00
Dmitry Werner 21e97e3c10
MINOR: Fix typo in AssignmentsManager (#20059)
CI / build (push) Waiting to run Details
Disonnected -> Disconnected

Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-30 01:30:25 +08:00
TaiJuWu bd14ed21b4
KAFKA-18486 Remove ReplicaManager#becomeLeaderOrFollower (#20037)
The PR do following:

1. Remove  ReplicaManager#becomeLeaderOrFollower.
2. Remove `LeaderAndIsrRequest` and `LeaderAndIsrResponse`
3. Migrate `LeaderAndIsrRequest.PartitionState` to server-common module
and change to `PartitionState`
4. Remove `ControllerEpoch` from PartitionState
5. Remove `isShuttingDown` from BrokerServer and ReplicaManager

Reviewers: Kuan-Po Tseng <brandboat@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-30 01:20:49 +08:00
Xuan-Zhang Gong 7d34cb4412
MINOR: Correct ProducerFailureHandlingTest Typo, Use waitForTopic Instead of waitForDeletion (#20054)
from https://github.com/apache/kafka/pull/19319#discussion_r2169555430

This pull request addresses a minor typo in the
ProducerFailureHandlingTest within the Apache Kafka project.
Specifically, it corrects an erroneous method call where waitForDeletion
was used instead of waitForTopic (or createTopic).

Reviewers: PoAn Yang <payang@apache.org>, TaiJuWu <tjwu1217@gmail.com>,
 Jhen-Yung Hsu <jhenyunghsu@gmail.com>, TengYao Chi
 <kitingiao@gmail.com>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-30 00:53:36 +08:00
TaiJuWu a95522a5ba
KAFKA-19042 Rewrite ConsumerBounceTest by Java (#19822)
This PR does the following:
1) Rewrites consumerBounceTest in Java.
2) Moves the test to clients-integration-test.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-06-30 00:40:36 +08:00
Dmitry Werner 959da32f97
MINOR: Use primitive data types in tools (#20038)
Use primitive data types instead of wrappers in the tools module.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-06-29 23:51:28 +08:00
k-apol a5a54dc32b
KAFKA-10357: Add new Kafka Streams exception classes (#19988)
CI / build (push) Waiting to run Details
Add new exception classes as defined in KIP-698.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2025-06-28 10:37:04 -07:00
Xuan-Zhang Gong 05b6e81688
KAFKA-19420 Don't export SocketServer from ClusterInstance (#20002)
CI / build (push) Waiting to run Details
Fixup PR Labels / fixup-pr-labels (needs-attention) (push) Has been cancelled Details
Fixup PR Labels / fixup-pr-labels (triage) (push) Has been cancelled Details
Docker Image CVE Scanner / scan_jvm (3.7.2) (push) Has been cancelled Details
Docker Image CVE Scanner / scan_jvm (3.8.1) (push) Has been cancelled Details
Docker Image CVE Scanner / scan_jvm (3.9.1) (push) Has been cancelled Details
Docker Image CVE Scanner / scan_jvm (4.0.0) (push) Has been cancelled Details
Docker Image CVE Scanner / scan_jvm (latest) (push) Has been cancelled Details
Fixup PR Labels / needs-attention (push) Has been cancelled Details
Refactor the code related to SocketServer  SocketServer is an internal
class, and normally the integration tests should not use it directly.
[KAFKA-19239](https://issues.apache.org/jira/browse/KAFKA-19239) will
add a new helper to expose the bound ports, and so the tests that need
to send raw request can leverage it without accessing the SocketServer.

Reviewers: PoAn Yang <payang@apache.org>, Ken Huang
 <s7133700@gmail.com>, TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-27 21:12:57 +08:00
Ken Huang b919836551
KAFKA-17662: config.providers configuration missing from the docs (#18930)
Ensure the config.providers configuration is documented for all
components supporting it

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Greg Harris
<gharris1727@gmail.com>, Matthias J. Sax <mjsax@apache.org>
2025-06-27 14:13:55 +02:00
David Jacot 202e216a60
MINOR: Cleanup hasMemberSubscriptionChanged (#20047)
CI / build (push) Waiting to run Details
Cleanup hasMemberSubscriptionChanged.
Remove unused InvalidRegularExpressionException from the signature.

Reviewers: Sean Quah <squah@confluent.io>, Chia-Ping Tsai
 <chia7712@gmail.com>, TengYao Chi <frankvicky@apache.org>
2025-06-27 14:16:49 +08:00
Matthias J. Sax 53faeb85ef
MINOR: Improve ProcessorContext JavaDocs (#20042)
CI / build (push) Waiting to run Details
Clarify that state stores are sharded, and shards cannot be shared
across Processors.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2025-06-26 10:06:46 -07:00
Apoorv Mittal 96ef1c520a
KAFKA-19436: Restrict cache update for ongoing batch/offset state (#20041)
CI / build (push) Waiting to run Details
In the stress testing it was noticed that on acquisition lock timeout,
some offsets were not found in the cache. The cache can be tried to be
updated in different acknowledgement calls hence if there is an ongoing
transition which is not yet finished but another parallel
acknowledgement triggers the cache update then the cache can be updated
incorrectly, while first transition is not yet finished.

Though the cache update happens for Archived and Acknowldeged records
hence this issue or existing implementation should not hamper the queues
functionality. But it might update the cache early when persister call
might fail or this issue triggers error logs with offset not found in
cache when acquisition lock timeouts (in some scenarios).

Reviewers: Abhinav Dixit <adixit@confluent.io>, Andrew Schofield
 <aschofield@confluent.io>
2025-06-26 15:08:15 +01:00
David Jacot 293043de9a
MINOR: Add group-coordinator label for pull requests (#20043)
This patch adds the group-coordinator label for pull requests.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2025-06-26 06:53:47 -07:00
David Jacot f6a78c4c2b
KAFKA-19246; OffsetFetch API does not return group level errors correctly with version 1 (#19704)
The OffsetFetch API does not support top level errors in version 1.
Hence, the top level error must be returned at the partition level.

Side note: It is a tad annoying that we create error response in
multiple places (e.g. KafkaApis, Group CoordinatorService). There were a
reason for this but I cannot remember.

Reviewers: Dongnuo Lyu <dlyu@confluent.io>, Sean Quah <squah@confluent.io>, Ken Huang <s7133700@gmail.com>, TengYao Chi <frankvicky@apache.org>
2025-06-26 06:29:43 -07:00
Lan Ding d7745f55d9
MINOR: Clean up in ReassignPartitionsCommandTest (#20035)
see https://github.com/apache/kafka/pull/20003#discussion_r2164659673

Replace `SimpleImmutableEntry` by `Map.entry`.  Replace following
out-of-date APIs:

    Arrays.asList      Collections.singleton
Collections.singletonList      Collections.singletonMap

Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, TengYao Chi
 <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-06-26 20:58:00 +08:00
Sanskar Jhajharia 56aeaa4c44
MINOR: Cleanup ShareFetchAcknowledgeRequestTest (#19852)
CI / build (push) Waiting to run Details
Now that Kafka supports Java 17, this PR cleans up the
ShareFetchAcknowledgeRequestTest.
The changes mostly include:
- Collections.singletonList() is replaced with List.of()
- Get rid of all asJava conversions

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-26 13:56:18 +08:00
Hong-Yi Chen dc82c766fa
KAFKA-18834 Fix LoggingResourceTest#testSetLevelDefaultScope (#19920)
- Flaky behavior
`LoggingResourceTest#testSetLevelDefaultScope` sometimes fails by not
capturing its expected WARN log.

- Root cause
Both `LoggersTest#testSetLevelWithValidRootLoggerNames` and
`LoggingResourceTest#testSetLevelDefaultScope` may share the same
`LoggerContext` when executed in the same JVM.
`LoggersTest#testSetLevelWithValidRootLoggerNames` calls
`loggers.setLevel("", ERROR)`, which mutates the global root logger
level to ERROR and suppresses WARN logs, which causes subsequent tests
to fail to emit WARN-level output.

- Fix in this PR
Resets the Log4j configuration after each test in `LoggersTest`,
ensuring that any global changes are reverted.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-06-26 13:39:08 +08:00
Mahsa Seifikar 7aaba96cc1
KAFKA-19282: Update quotaTypesEnabled on quota removal in ClientQuotaManager (#19742)
CI / build (push) Waiting to run Details
In `kafka.server.ClientQuotaManager` class, `quotaTypesEnabled` is not updated when a quota is removed via `removeQuota` method in `DefaultQuotaCallback` class. This field is set when quotas are added in `updateQuota` but it's never changed or cleared. So in case all the quotas have been removed dynamically, the system may incorrectly assume the quotas are active, which leads to unnecessary metric creation or updates until the broker is restarted.

Reviewers: Jonah Hooper <jhooper@confluent.io>, Hailey Ni <hni@confluent.io>, Alyssa Huang <ahuang@confluent.io>, David Jacot <djacot@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>
2025-06-25 21:29:46 +01:00
Lucas Brutschy 23ddb1d8ac
MINOR: Reject requests using unsupported features in KIP-1071 (#20031)
CI / build (push) Waiting to run Details
KIP-1071 does not currently support all features planned in the KIP. We
should reject any requests that are using features that are currently
not implemented.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Matthias J. Sax
 <matthias@confluent.io>, Bill Bejeck <bill@confluent.io>
2025-06-25 14:48:56 +02:00
Jing-Jia Hung 5e23df0c8d
KAFKA-18486 Migrate tests to use applyDelta instead of becomeLeaderOrFollower for testInconsistentIdReturnsError and others (#20014)
continues the migration effort for KAFKA-18486 by replacing usage of the
deprecated `becomeLeaderOrFollower` API with `applyDelta` in several
test cases.

#### Updated tests:
- `testInconsistentIdReturnsError`
- `testMaybeAddLogDirFetchers`
- `testMaybeAddLogDirFetchersPausingCleaning`
- `testSuccessfulBuildRemoteLogAuxStateMetrics`
- `testVerificationForTransactionalPartitionsOnly`
- `testBecomeFollowerWhenLeaderIsUnchangedButMissedLeaderUpdate`

Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, TaiJuWu
 <tjwu1217@gmail.com>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-25 20:02:27 +08:00
Rajini Sivaram 33a1648c44
MINOR: Fix response for consumer group describe with empty group id (#20030)
ConsumerGroupDescribe with an empty group id returns a response containing `null` groupId in a non-nullable field. Since the response cannot be serialized, this results in UNKNOWN_SERVER_ERROR being returned to the client. This PR sets the group id in the response to an empty string instead and adds request tests for empty group id.

Reviewers: David Jacot <djacot@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2025-06-25 10:33:44 +01:00
Yu-Syuan Jheng 2d943419a3
MINOR: Remove unused DescribeTopicPartitionResponse.prepareResponse function (#20032)
The prepareResponse function is no longer used anywhere in the codebase.
Removing unused code improves maintainability and readability.

Reviewers: PoAn Yang <payang@apache.org>, Yung
<yungyung7654321@gmail.com>, TengYao Chi <frankvicky@apache.org>, Ken
Huang <s7133700@gmail.com>
2025-06-25 17:26:22 +08:00
Matthias J. Sax 4387132926
KAFKA-19398: (De)Register oldest-iterator-open-since-ms metric dynamically (#20022)
CI / build (push) Waiting to run Details
The metric for oldest-iterator-open-since-ms might report a null value
if there is not open iterator.

This PR changes the behavior to dynamically register/deregister the
entire metric instead of allowing it to return a null value.

Reviewers: Bill Bejeck <bbejeck@apache.org>
2025-06-24 17:01:36 -07:00
Colin Patrick McCabe 6b2013a001
KAFKA-19294: Fix BrokerLifecycleManager RPC timeouts (#19745)
CI / build (push) Waiting to run Details
Previously, we could wait for up to half of the broker session timeout
for an RPC to complete, and then delay by up to half of the broker
session timeout. When taken together, these two delays could lead to
brokers erroneously missing heartbeats.

This change removes exponential backoff for heartbeats sent from the
broker to the controller. The load caused by heartbeats is not heavy,
and controllers can easily time out heartbeats when the queue length is
too long. Additionally, we now set the maximum RPC time to the length of
the broker period. This minimizes the impact of heavy load.

Reviewers: José Armando García Sancio <jsancio@apache.org>, David Arthur <mumrah@gmail.com>
2025-06-24 16:23:25 -07:00
Calvin Liu dcc9320bf6
KAFKA-19383: Handle the deleted topics when applying ClearElrRecord (#19916)
https://issues.apache.org/jira/browse/KAFKA-19383  When applying the
ClearElrRecord, it may pick up the topicId in the image without checking
if the topic has been deleted. This can cause the creation of a new
TopicRecord with an old topic ID.

Reviewers: Alyssa Huang <ahuang@confluent.io>, Artem Livshits <alivshits@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
2025-06-24 13:26:35 -07:00
Lan Ding 51f2c7b2b6
MINOR: fix reassign command bug (#20003)
see

9570c67b8c/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala (L1208)
During the rewrite for
[KAFKA-14595](https://github.com/apache/kafka/pull/13247), the relevant
condition was omitted.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-06-25 02:34:13 +08:00
Alyssa Huang 3460ded386
KAFKA-19411: Fix deleteAcls bug which allows more deletions than max records per user op (#19974)
If there are more deletion filters after we initially hit the
`MAX_RECORDS_PER_USER_OP` bound, we will add an additional deletion
record ontop of that for each additional filter.

The current error message returned to the client is not useful either,
adding logic so client doesn't just get `UNKNOWN_SERVER_EXCEPTION` with
no details returned.
2025-06-24 11:24:50 -07:00
Ken Huang 023833fe1f
KAFKA-18778 Fix the inconsistent lastest supported version in StorageTool.scala and FutureCommand (#19157)
To maintain code consistency, `MetadataVersion#fromVersionString` uses
`latestTesting()` as the latest version. Therefore, in the tools, we
also need to maintain consistency by updating the outer logging to use
`latestTesting()`.

See the discussion:
https://github.com/apache/kafka/pull/18845#discussion_r1950706791

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-06-25 01:59:52 +08:00
Okada Haruki 959021de59
KAFKA-19407 Fix potential IllegalStateException when appending to timeIndex (#19972)
## Summary
- Fix potential race condition in
LogSegment#readMaxTimestampAndOffsetSoFar(), which may result in
non-monotonic offsets and causes replication to stop.
- See https://issues.apache.org/jira/browse/KAFKA-19407 for the details
how it happen.

Reviewers: Vincent PÉRICART <mauhiz@gmail.com>, Jun Rao
 <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-06-25 00:35:53 +08:00
Alieh Saeedi d61b162b13
KAFKA-19244: Fix flaky streams offsets reset integration test (#20027)
CI / build (push) Waiting to run Details
This PR deflakes the
`testResetOffsetsWithDeleteSpecifiedInternalTopics()`

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2025-06-24 14:45:39 +02:00
Apoorv Mittal 1ca8779bee
MINOR: Correcting client error for fenced share partition (#20023)
Correct the error when SharePartition is fenced.

Reviewers: Abhinav Dixit <adixit@confluent.io>, Sushant Mahajan
 <smahajan@confluent.io>, Andrew Schofield <aschofield@confluent.io>
2025-06-24 09:46:14 +01:00
Sushant Mahajan 3d4407ff9d
MINOR: Change exceptions for few error codes in SharePartition. (#20020)
CI / build (push) Waiting to run Details
* The `SharePartition` class wraps the errors received from
`PersisterStateManager` to be sent to the client.
* In this PR, we are categorizing the errors a bit better.
* Some exception messages in `PersisterStateManager` have been updated
to show the share partition key.
* Tests have been updated wherever needed.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal
 <apoorvmittal10@gmail.com>
2025-06-23 19:27:15 +01:00
Lucas Brutschy 4fedffd282
KAFKA-19429: Deflake streams_smoke_test (#20019)
It looks like we are checking for properties that are not guaranteed
under at_least_once, for example, exact counting (not allowing for
overcounting).

This change relaxes the validation constraint to only check that we
counted _at least_ N messages, and our sums come out as _at least_ the
expected sum.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2025-06-23 20:16:52 +02:00
Lucas Brutschy 261e861340
KAFKA-19422: Deflake streams_application_upgrade_test (#20004)
In this upgrade test, applications sometimes crash before the upgrade,
so it's actually triggering a bug in several older versions (2.x and
possibly others). It seems to be a rare race condition that has been
happening since 2022. Since we are not going to roll out a patch release
for Kafka Streams 2.x, we should just allow applications to crash before
the upgrade.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2025-06-23 17:41:05 +02:00
Sushant Mahajan cb809e2574
MINOR: Change snapshot epoch type to int32 in schema. (#20016)
CI / build (push) Waiting to run Details
* `SnapshotEpoch` type in `ShareSnapshotValue.json` and
`ShareUpdateValue.json` is currently
`uint16` which might overflow under heavy traffic.
* To be consistent with other epochs, this PR updates the type to
`int32`.

Reviewers: Andrew Schofield <aschofield@confluent.io>, ShivsundarR
 <shr@confluent.io>
2025-06-23 14:15:01 +01:00
Ming-Yen Chung b38573fcaa
KAFKA-18486 Remove becomeLeaderOrFollower from testPartition*, testPreferredReplicaAs* (#20009)
Replace `leaderAndIsrRequest` and `becomeLeaderOrFollower` with
`TopicsDelta`, `MetadataImage` and `ReplicaManager#applyDelta` for the
following tests:
* testPartitionListener
* testPartitionMarkedOfflineIfLogCantBeCreated
* testPartitionMetadataFileNotCreated
* testPartitionsWithLateTransactionsCount
* testPreferredReplicaAsFollower
* testPreferredReplicaAsLeader
* testPreferredReplicaAsLeaderWhenSameRackFollowerIsOutOfIsr
* testProducerIdCountMetrics

Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-23 16:42:30 +08:00
Lucas Brutschy 4377ee0808
KAFKA-19421: Deflake streams_broker_down_resilience_test (#19999)
`streams_broker_down_resilience_test` produce messages with `null` key
to a topic with three partitions and expect each partition to be
non-empty afterward. But I don't think this is a correct assumption, as
a producer may try to be sticky and only produce to two partitions.

This cause occasional flakiness in the test.

The fix is to produce records with keys.

Reviewers: Matthias J. Sax <matthias@confluent.io>, PoAn Yang
 <payang@apache.org>
2025-06-23 10:30:13 +02:00
Sushant Mahajan 56a6ba2d2e
MINOR: Add retention prop to share group state topic. (#20013)
CI / build (push) Waiting to run Details
*

https://cwiki.apache.org/confluence/display/KAFKA/KIP-932%3A+Queues+for+Kafka
states the `retention.ms` property for the `__share_group_state` to be
`-1`.
* This PR makes it explicit when defining the values of those configs.
* Existing test has been updated.

```
$ bin/kafka-topics.sh --bootstrap-server localhost:9092 --describe
--topic __share_group_state

Topic: __share_group_state      TopicId: XCwzZjEGSjm5lUc_BeCrqA
PartitionCount: 50      ReplicationFactor: 1
Configs:
compression.type=producer,
min.insync.replicas=1,
cleanup.policy=delete,
segment.bytes=104857600,
retention.ms=-1
...
```

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-06-22 22:02:35 +01:00