Flaky Test Report / Flaky Test Report (push) Has been cancelledDetails
- Metadata doesn't have the full view of topicNames to ids during
rebootstrap of client or when topic has been deleted/recreated. The
solution is to pass down topic id and stop trying to figure it out later
in the logic.
---------
Co-authored-by: Kirk True <kirk@kirktrue.pro>
When the group coordinator is processing a heartbeat from a share
consumer, it must decide whether the recompute the assignment. Part of
this decision hinges on whether the assigned partitions match the
partitions initialised by the share coordinator. However, when the set
of subscribed topics changes, there may be initialised partitions which
are not currently assigned. Topics which are not subscribed should be
omitted from the calculation about whether to recompute the assignment.
Co-authored-by: Sushant Mahajan <smahajan@confluent.io>
Reviewers: Lan Ding <53332773+DL1231@users.noreply.github.com>, Sushant
Mahajan <smahajan@confluent.io>, Apoorv Mittal
<apoorvmittal10@gmail.com>
Finalise the share group SimpleAssignor for heterogeneous subscriptions.
The assignor code is much more accurate about the number of partitions
assigned to each member, and the number of members assigned for each
partition. It eliminates the idea of hash-based assignment because that
has been shown to the unhelpful. The revised code is very much more
effective at assigning evenly as the number of members grows and shrinks
over time.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
Update the outdated Javadocs in Metrics.java. The `MetricName(String
name, String group)` constructor in MetricName.java was removed in
59b918ec2b
Minor typo fixes included.
Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi
<kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
* There are instances where share group delete calls in group
coordinator (`onPartitionsDelete`, `deleteShareGroups`) where we lookup
the metadata image to fetch the topic id/partitions/topic name for a
topic name/id. However, there have been
instances where the looked up info was not found due to cluster being
under load or the underlying topic being deleted and information not
propagated correctly.
* To remedy the same, this PR adds checks to determine that topic is
indeed present in the image before the lookups thus preventing NPEs. The
problematic situations are logged.
* New tests have been added for `GroupMetadataManger` and
`GroupCoordinatorService`.
Reviewers: Andrew Schofield <aschofield@confluent.io>
While testing the code in https://github.com/apache/kafka/pull/19820, it
became clear that the error handling problems were due to the underlying
Admin API. This PR fixes the error handling for top-level errors in the
AlterShareGroupOffsets RPC.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Lan Ding
<isDing_L@163.com>, TaiJuWu <tjwu1217@gmail.com>
Estimate the fetch size for remote fetch to avoid to exceed the
`fetch.max.bytes` config. We don't want to query the remoteLogMetadata
during API handling, thus we assume the remote fetch can get
`max.partition.fetch.bytes` size. Tests added.
Reviewers: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
### About
`nextFetchOffset` function in `SharePartition` updates the fetch offsets
without considering batches/offsets which might be undergoing state
transition. This can cause problems in updating to the right fetch
offset.
### Testing
The new code added has been tested with the help of unit tests.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
issue: https://github.com/apache/kafka/pull/19687/files#r2094574178
Why:
- To improve performance by avoiding redundant temporary collections and
repeated method calls.
- To make the utility more flexible for inputs from both Java and Scala.
What:
- Refactored `createResponseConfig` in `ConfigHelper.scala` by
overloading the method to accept both Java maps and `AbstractConfig`.
- Extracted helper functions to `ConfigHelperUtils` in the server
module.
Reviewers: Ken Huang <s7133700@gmail.com>, Jhen-Yung Hsu
<jhenyunghsu@gmail.com>, TengYao Chi <kitingiao@gmail.com>, Chia-Ping
Tsai <chia7712@gmail.com>
Now that Kafka supports Java 17, this PR makes some changes in
jmh-benchmarks module. The changes mostly include:
- Collections.emptyList(), Collections.singletonList() and
Arrays.asList() are replaced with List.of()
- Collections.emptyMap() and Collections.singletonMap() are replaced
with Map.of()
- Collections.singleton() is replaced with Set.of()
Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Ken Huang
<s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
* 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>
* 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>
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>
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>
* 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>
* 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>
- 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>
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.
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>
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>
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>
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>
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>
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>
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>
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>
Fixup PR Labels / fixup-pr-labels (needs-attention) (push) Has been cancelledDetails
Fixup PR Labels / fixup-pr-labels (triage) (push) Has been cancelledDetails
Docker Image CVE Scanner / scan_jvm (3.7.2) (push) Has been cancelledDetails
Docker Image CVE Scanner / scan_jvm (3.8.1) (push) Has been cancelledDetails
Docker Image CVE Scanner / scan_jvm (3.9.1) (push) Has been cancelledDetails
Docker Image CVE Scanner / scan_jvm (4.0.0) (push) Has been cancelledDetails
Docker Image CVE Scanner / scan_jvm (latest) (push) Has been cancelledDetails
Fixup PR Labels / needs-attention (push) Has been cancelledDetails
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>
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>
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>
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>
This patch adds the group-coordinator label for pull requests.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
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>
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>
- 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>
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>
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>
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>
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>
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>
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>
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>
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>
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.