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>
Extending test coverage of authorization for streams group RPC
StreamsGroupDescribe. The RPC requires DESCRIBE GROUP and DESCRIBE TOPIC
permissions for all topics.
Reviewers: Bill Bejeck <bbejeck@apache.org>
These can be used to implement transformations on top of the RPC
definitions. Group IDs were already marked. This PR additionally adds
the entityType for all topic names.
Reviewers: Matthias J. Sax <matthias@confluent.io>
This is a follow up to [https://github.com/apache/kafka/pull/19910](url)
`The coordinator failed to write an epoch fence transition for producer
jt142 to the transaction log with error COORDINATOR_NOT_AVAILABLE. The
epoch was increased to 2 but not returned to the client
(kafka.coordinator.transaction.TransactionCoordinator)` -- as we don't
bump the epoch with this change, we should also update the message to
not say "increased" and remove the
**epochAndMetadata.transactionMetadata.hasFailedEpochFence = true** line
In the test, the expected behavior is:
1) First append transaction to the log fails with
COORDINATOR_NOT_AVAILABLE (epoch 1)
2) We try init_pid again, this time the SINGLE epoch bump succeeds, and
the following things happen simultaneously (epoch 2)
-> Transition to COMPLETE_ABORT
-> Return CONCURRENT_TRANSACTION error to the client
3) The client retries, and there is another epoch bump; state
transitions to EMPTY (epoch 3)
Reviewers: Justine Olshan <jolshan@confluent.io>, Artem Livshits
<alivshits@confluent.io>
This change compares the remote replica's HWM with the leader's HWM and
completes the FETCH request if the remote HWM is less than the leader's
HWM. When the leader's HWM is updated any pending FETCH RPC is
completed.
Reviewers: Alyssa Huang <ahuang@confluent.io>, David Arthur
<mumrah@gmail.com>, Andrew Schofield <aschofield@confluent.io>
`CreateTopicPolicy#validate` may throw unexpected exception other than
`PolicyViolationException`. We should handle this case as well.
Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
Use Java to rewrite ProducerSendWhileDeletionTest by new test infra and
move it to client-integration-tests module.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
see https://github.com/apache/kafka/pull/19948#discussion_r2150617216,
replace test "catch exception" by assertThrows.
Reviewers: PoAn Yang <payang@apache.org>, Ken Huang
<s7133700@gmail.com>, TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
Extending test coverage of authorization for streams group RPC
StreamsGroupHeartbeat. The RPC requires READ GROUP and DESCRIBE TOPIC
permissions for all topics. For creating internal topics, we require
CREATE TOPIC permission. If internal topic creation fails, the request
does not fail, but the status reflects this problem.
Reviewers: Bill Bejeck <bbejeck@apache.org>
In [KIP-1143](https://cwiki.apache.org/confluence/x/LwqWF), it
deprecated Endpoint#listenerName and removed
org.apache.kafka.network.EndPoint. Certain parts of the code depend on
listener name normalization. We should add a test to make sure there is
no regression.
Followup:
https: //github.com/apache/kafka/pull/19191#issuecomment-2939855317
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Clarify the interaction of `min.insync.replicas` and `ack=all`
configuration. Prior to this change, the doc for `min.insync.replicas`
could have been interpreted as being used to short-circuit in the
`acks=all` case as if it would be enough if `min.inscyn.replicas`
number of brokers replicated a message before it can be acknowledged
back to the producer.
Reviewers: Matthias J. Sax <matthias@confluent.io>
---------
Co-authored-by: Matthias J. Sax <mjsax@apache.org>
The included tests are as follows:
- testFencedErrorCausedByBecomeLeader
- testFetchBeyondHighWatermark
- testFetchFollowerNotAllowedForOlderClients
- testFetchFromFollowerShouldNotRunPreferLeaderSelect
- testFetchFromLeaderAlwaysAllowed
- testFetchMessagesWhenNotFollowerForOnePartition
- testFetchRequestRateMetrics
- testFetchShouldReturnImmediatelyWhenPreferredReadReplicaIsDefined
- testFollowerFetchWithDefaultSelectorNoForcedHwPropagation
- testFollowerStateNotUpdatedIfLogReadFails
I removed `testFetchMessagesWithInconsistentTopicId ` as it's no longer
needed, the "topicId" is now mandatory and cannot be null in our new
implementation.
Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Lan Ding
<isDing_L@163.com>, Chia-Ping Tsai <chia7712@gmail.com>
This PR fixes a typo in the `RocksDBStore` where it currently uses a
getter named `optimizeFiltersForHits`. However, to set the flag to true,
we have to use `setOptimizeFiltersForHits(true)` instead.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Chia-Ping Tsai
<chia7712@gmail.com>
Change becomeLeaderOrFollower to applyDelta in following test cases
- testReadCommittedFetchLimitedAtLSO
- testReceiveOutOfOrderSequenceExceptionWithLogStartOffset
- testRemoteFetchExpiresPerSecMetric
- testRemoteLogReaderMetrics
Reviewers: PoAn Yang <payang@apache.org>, Bolin Lin
<linbolin1230@gmail.com>, Yung <yungyung7654321@gmail.com>, Jimmy Wang
<48462172+JimmyWang6@users.noreply.github.com>, Ken Huang
<s7133700@gmail.com>, TengYao Chi <frankvicky@apache.org>
I noticed that in commit
[a662bc56](a662bc5634),
renamed ignored exceptions `e` to `swallow`.
Here's a small patch to make that change consistent in other files:
AsyncKafkaConsumer.java, KafkaConsumerTest.java,
AsyncKafkaConsumerTest.java
Reviewers: Kirk True <kirk@kirktrue.pro>, Lan Ding <isDing_L@163.com>,
Chia-Ping Tsai <chia7712@gmail.com>
Replace `becomeLeaderOrFollower` with `applyDelta` in method
RemoteFetchExpiresPerSecMetric and RemoteLogReaderMetrics
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Change `becomeLeaderOrFollower` to `applyDelta` in following test cases
* testOffsetOutOfRangeExceptionWhenFetchMessages
* testOffsetOutOfRangeExceptionWhenReadFromLog
* testOldFollowerLosesMetricsWhenReassignPartitions
* testOldLeaderLosesMetricsWhenReassignPartitions
Reviewers: Bolin Lin <linbolin1230@gmail.com>, Lan Ding
<isDing_L@163.com>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
Upgraded RocksDB from 9.7.3 to 10.1.3, deprecate two configuration in
`RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter.java`
- random_access_max_buffer_size (removed since v9.11.1
541761eaaa)
- rate_limiter (deprecated since v7.6.0
25cc564ff7)
Add one configuration:
- daily_offpeak_time_utc (introduced since v9.11.1
9b1d0c02e9)
Reviewers: Bruno Cadonna <cadonna@apache.org>
Deprecate the `remote.log.manager.thread.pool.size` configuration and introduce a new dynamic configuration:
`remote.log.manager.follower.thread.pool.size`.
Reviewers: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Luke Chen <showuon@gmail.com>
Minor fix to correct the validate condition for GetTelemetryRequests.
Added respective tests as well.
Reviewers: Andrew Schofield <aschofield@confluent.io>
Adding a descriptive comment why it's necessary to filter metrics
registration by thread-id tags. This is due to the fact that the
`StreamsMetric` is a singleton shared by all StreamThread instances, so
we need to make sure only add metrics for the current StreamThread
otherwise duplicate metrics are registered.
Reviewers: Matthias Sax <mjsax@apache.org>
Adds a test dependency on
[mock-oauth2-server](https://github.com/navikt/mock-oauth2-server/) for
integration tests for OAuth layer. Also includes fixes for some
regressions that were caught by the integration tests.
Reviewers: Manikumar Reddy <manikumar@confluent.io>, Lianet Magrans
<lmagrans@confluent.io>
When InitProducerId is handled on the transaction coordinator, the
producer epoch is incremented (so that we fence stale requests), then if
a transaction was ongoing during this time, it's aborted. With
transaction version 2 (a.k.a. KIP-890 part 2), abort increments the
producer epoch again (it's the part of the new abort / commit protocol),
so the epoch ends up incremented twice.
In most cases, this is benign, but in the case where the epoch of the
ongoing transaction is 32766, it's incremented to 32767, which is the
maximum value for short. Then, when it's incremented for the second
time, it goes negative, causing an illegal argument exception.
To fix this we just avoid bumping the epoch a second time.
Reviewers: Justine Olshan <jolshan@confluent.io>, Artem Livshits
<alivshits@confluent.io>
This PR uses topic IDs received in assignment (under new protocol) to
ensure that only these assigned topics are included in the consumer
metadata requests performed when the user subscribes to broker-side
regex (RE2J).
For handling the edge case of consumer needing metadata for topics IDs
(from RE2J) and topic names (from transient topics), the approach is to
send a request for the transient topics needed temporarily, and once
those resolved, the request for the topic IDs needed for RE2J will
follow. (this is because the broker doesn't accept requests for names
and IDs at the same time)
With the changes we also end up fixing another issue (KAFKA-18729) aimed
at avoiding iterating the full set of assigned partitions when checking
if a topic should be retained from the metadata response when using
RE2J.
Reviewers: David Jacot <djacot@confluent.io>
Fixed a long-standing issue where the client JWT validation was decoding
the JWT sections using base 64 instead of URL-safe base 64.
Note: server-side validation leverages the jose4j library for parsing
JWTs, hence no fix is needed there.
Reviewers: Lianet Magrans <lmagrans@confluent.io>, Manikumar Reddy
<manikumar@confluent.io>
---------
Co-authored-by: Lianet Magrans <98415067+lianetm@users.noreply.github.com>
Consumers can subscribe to an RE2J SubscriptionPattern that will be
resolved and maintained on the server-side (KIP-848). Currently, those
regexes are refreshed on the coordinator when a consumer subscribes to a
new regex, or if there is a new topic metadata image (to ensure regex
resolution stays up-to-date with existing topics)
But with
[KAFKA-18813](https://issues.apache.org/jira/browse/KAFKA-18813), the
topics matching a regex are filtered based on ACLs. This generates a new
situation, as regexes resolution do not stay up-to-date as topics become
visible (ACLs added/delete).
This patch introduces time-based refresh for the subscribed regex by
- Adding internal `group.consumer.regex.batch.refresh.max.interval.ms`
config
that controls the refresh interval.
- Schedule a regex refresh when updating regex subscription if the
latest refresh is older than the max interval.
Reviewers: David Jacot <djacot@confluent.io>