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.
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>
## 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>
Correct the error when SharePartition is fenced.
Reviewers: Abhinav Dixit <adixit@confluent.io>, Sushant Mahajan
<smahajan@confluent.io>, Andrew Schofield <aschofield@confluent.io>
* 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>
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>
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>
* `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>
`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>
*
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>
Description
* Replace `org.apache.kafka.common.test.TestUtils` with
`org.apache.kafka.test.TestUtils` in outer package modules to
standardize test utility usage
* Move `waitUntilLeaderIsElectedOrChangedWithAdmin` method from
`org.apache.kafka.test.TestUtils` to `ClusterInstance` and refactor for
better code organization
* Add `org.apache.kafka.test.TestUtils` dependency to
`transaction-coordinator` import control
Reviewers: PoAn Yang [payang@apache.org](mailto:payang@apache.org), Ken
Huang [s7133700@gmail.com](mailto:s7133700@gmail.com), Ken Huang
[s7133700@gmail.com](mailto:s7133700@gmail.com), Chia-Ping Tsai
[chia7712@gmail.com](mailto:chia7712@gmail.com)
this is a follow-up for https://github.com/apache/kafka/pull/19685
The timeout issue in `AsyncConsumer#unsubscribe` was fixed by
https://github.com/apache/kafka/pull/19779. As a result, the test
`GroupAuthorizerIntegrationTest#testConsumeUnsubscribeWithoutGroupPermission`
should now retain its original behavior as expected prior to the issue.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
In KRaft, custom KafkaPrincipalBuilder instances must implement
KafkaPrincipalSerde to support the forward mechanism. Currently, this
requirement is not enforced and relies on the developer's attention.
With this patch, we can prevent incorrect implementations at compile
time.
Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi
<kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
The old approach `OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG` will override
`OFFSETS_TOPIC_PARTITIONS_CONFIG` config, this behaviour is not
expected, we should fix it.
Reviewers: TengYao Chi <frankvicky@apache.org>
Now that Kafka Brokers support Java 17, this PR makes some changes in
core module. The changes in this PR are limited to only some Scala files
in the Core module's tests. 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()
To be clear, the directories being targeted in this PR from unit.kafka
module:
- admin
- cluster
- coordinator
- docker
- integration
- metrics
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
The test testShareFetchRequestSuccessfulSharingBetweenMultipleConsumers
was recently found to be flaky. Making the following small change that
could potentially resolve the issue. Earlier, 1000 records were being
produced and then 3 consecutive share fetch requests were being sent. At
the end, assertions were done to make sure each share consumer receives
some records, and that none of them consume the same record. Since the
motive for the test is to see if multiple consumers can share the same
subscription and not consume the same record, a better way would be to
produce a record, consume that and repeat it 3 times with the 3
consumers. This ensures that every consumer consume a record, and a
previously consume record is not consumed again by the subsequent share
fetches.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Andrew Schofield
<aschofield@confluent.io>
* The share group rebalance metric was not being invoked at the
appropriate group id bump position.
* This PR solves the issue.
* The metric name has been updated
(s/rebalance-rate/share-group-rebalance-rate,
s/rebalance-count/share-group-rebalance-count/)
* Updated tests in `GroupMetadataManagerTest` and
`GroupCoordinatorMetricsTest`
Reviewers: Andrew Schofield <aschofield@confluent.io>
The test is resizing the `__consumer_offset` topic after broker start.
This seems to be completely unsupported. The group coordinator fetches
the number of partitions for the consumer offset topic once and never
updates it. So we can be in a state where two brokers have a different
understanding of how `__consumer_offsets` are partitioned.
The result in this test can be that two group coordinators both think
they own a certain group. The test is resizing `__consumer_offsets`
right after start-up from 3 to 50. Before the broker bounce, the GC
operates on only three partitions (0-2). During the bounce, we get new
brokers that operate on (0-49). This means that two brokers can both
think, at the same time, that they own a group.
Reviewers: Matthias J. Sax <matthias@confluent.io>
Finalise the share group SimpleAssignor for homogeneous 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.
A future PR will address the code for heterogeneous subscriptions.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
Fix to avoid flakiness in verifiable producer system test. The test
lists running processes and greps to find the VerifiableProducer one,
but wasn't providing an specific pattern to grep (so flaky if there were
more than one process containing the default grep pattern "kafka")
Fix by passing a "proc_grep_string" to filter when looking for the
VerifiableProducer process.
All test pass successfully after the change.
Reviewers: PoAn Yang <payang@apache.org>, Andrew Schofield
<aschofield@confluent.io>
* Few logs in `PersisterStateManager` were noisy and not adding much
value.
* For the sake of reducing pollution, they have been moved to debug
level.
* Additional debug log in `DefaultStatePersister` to track epochs.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Andrew Schofield
<aschofield@confluent.io>
Replace `singleton` with `Set.of` in the SubscriptionStateTest.
Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
<payang@apache.org>, Yung <yungyung7654321@gmail.com>, TengYao Chi
<frankvicky@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
MetadataRequest and MetadataResponse version 0 is still supported.
Remove from README.md mentioning that they're not supported. It's
unclear how such a mention was ever included in the file. It has always
been there
Reviewers: Andrew Schofield <aschofield@confluent.io>
The protocol type; for Connect assignors this is "eager", "compatible",
or "sessioned"
Since `ConnectAssignor` is an interface and the protocol parameter is
restricted to "eager", "compatible", or "sessioned", it aligns with the
existing ConnectProtocolCompatibility enum. Therefore, we can update the
code to use `ConnectProtocolCompatibility` directly.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Instead of using the GH CLI (which requires some local setup), we can
just use cURL and JQ. Additionally, the original logic required setting
up alias.update-cache. After the change, this configuration is no longer
necessary.
validation on mac:
```
❯ ./update-cache.sh
Local branch 'trunk-cached' successfully updated to f26974b16d (from 19
hours ago).
```
Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
This PR rewrites the IntegrationTestUtils.java from Scala to Java.
## Changes:
- Converted all the existing Scala code in IntegrationTestUtils.scala
into Java in IntegrationTestUtils.java.
- Preserved the original logic and functionality to ensure backward
compatibility.
- Updated relevant imports and dependencies accordingly.
Motivation:
The rewrite aims to standardize the codebase in Java, which aligns
better with the rest of the project and facilitates easier maintenance
by the Java-centric team.
Reviewers: TaiJuWu <tjwu1217@gmail.com>, Ken Huang <s7133700@gmail.com>,
PoAn Yang <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
Configure tests in ShareConsumer to run exclusively in Raft-Isolated
mode to reduce execution time.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Jhen-Yung Hsu
<jhenyunghsu@gmail.com>, Ken Huang <s7133700@gmail.com>
*What*
- There was a possibility of a flake occurring in one of the tests in
`ShareConsumerTest` where we are not waiting for the assignment to be
received and synced before we start consuming on `poll()`.
- There should ideally be a `waitedPoll()` or a dedicated poll which
would trigger the reconcilation and updation of the assignment.
Scenario :
- We are doing 3 `polls` to receive 1500 records.
- On debugging, I noticed 1st poll always returned 0 records as this was
triggering reconcilation, the test passed locally as 2nd and 3rd polls
add up to give us 1500 records almost always (as 500 is a soft limit).
The test can fail if these 2 `polls` do not add up 1500. We need a
maximum
of 3 polls to receive 1500 records with `max.poll.records` set to 500.
- Now we have introduced a loop which would keep polling until we
receive the expected records and then close.
Reviewers: Andrew Schofield <aschofield@confluent.io>
BTW: whether we should rename
`ReplicaManager#updateLeaderAndFollowerMetrics`
Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
<payang@apache.org>, TengYao Chi <kitingiao@gmail.com>, Lan Ding
<isDing_L@163.com>, Chia-Ping Tsai <chia7712@gmail.com>
Add a note to the upgrade documentation indicating that
`remote.log.manager.thread.pool.size` has been deprecated. Users should
now use `remote.log.manager.follower.thread.pool.size` instead.
Reviewers: PoAn Yang <payang@apache.org>, Chia-Ping Tsai
<chia7712@gmail.com>, Yung <yungyung7654321@gmail.com>, Xuan-Zhang Gong
<gongxuanzhangmelt@gmail.com>, TengYao Chi <kitingiao@gmail.com>, Ken
Huang <s7133700@gmail.com>
Adds a new section **"Installation Preparation"** to the Kafka Docker
examples README. It documents a known issue with Docker versions prior
to 20.10.4, where directory permissions (such as `/opt/kafka/config`)
are not properly set, causing startup failures for non-root users (e.g.,
`appuser`).
This issue was fixed in Docker [20.10.4
release](https://docs.docker.com/engine/release-notes/20.10/#20104),
specifically in Use 0755 permissions when creating missing directories
[moby/moby#42017](https://github.com/moby/moby/pull/42017).
Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Lan Ding
<isDing_L@163.com>, Chia-Ping Tsai <chia7712@gmail.com>
---------
Co-authored-by: Jhen-Yung Hsu <yungyung7654321@gmail.com>
Fix to ensure assigned partitions whose topics are not in the consumer
explicit subscription are considered not fetchable (so that no records
are returned on poll for them)
This scenario could happen in the new async consumer (using the Consumer
rebalance protocol) when the subscription changes, because the consumer
will keep its assignment until the coordinator sends a new one (broker
drives assignments).
This does not happen in the classic consumer because the assignment
logic lives on the client-side, so the consumer pro-actively updates
assignment as needed.
This PR validates assignment vs subscription on fetch for explicit
subscription only. Regular expressions, shared subscription remain
unchanged (regex case still under discussion, will be handled separately
if needed)
Reviewers: Andrew Schofield <aschofield@confluent.io>, TengYao Chi
<frankvicky@apache.org>, Kirk True <ktrue@confluent.io>, Jhen-Yung Hsu
<jhenyunghsu@gmail.com>
Profiling has shown that using the Collections Streams API approach adds
unnecessary overhead compared to a traditional for loop. Minor revisions
to the code have been made to use simpler constructs to improve
performance.
Reviewers: Lianet Magrans <lmagrans@confluent.io>, Andrew Schofield
<aschofield@confluent.io>