*What*
We were missing a parantheses when we invoked a method
`supports_formatter_property()`. This would mean we would get the object
not call the function.
PR fixes this by including parantheses and invoking the actual function.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
document the behavior of "-1" (HIGH_WATERMARK)
Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi
<kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
*What*
https://issues.apache.org/jira/browse/KAFKA-19623
- The PR implements KIP-1147
(https://cwiki.apache.org/confluence/display/KAFKA/KIP-1147%3A+Improve+consistency+of+command-line+arguments)
for the console tools i.e. `ConsoleProducer`, `ConsoleConsumer` and
`ConsoleShareConsumer`.
- Currently the previous names for the options are still usable but
there will be warning message stating those are deprecated and will be
removed in a future version.
- I have added unit tests and also manually verified using the console
tools that things are working as expected.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Jhen-Yung Hsu
<jhenyunghsu@gmail.com>, Jimmy Wang
<48462172+JimmyWang6@users.noreply.github.com>
The format of the code in `CoordinatorLoaderImpl` in inconsistent with
the rest of the code in the package. This small PR fixes it.
Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi
<kitingiao@gmail.com>, Andrew Schofield <aschofield@confluent.io>, Sean
Quah <squah@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
This PR introduces an ExpiringErrorCache that temporarily stores topic
creation errors, allowing the system to provide detailed failure reasons
in subsequent heartbeat responses.
Key Designs:
Time-based expiration: Errors are cached with a TTL based on the
streams group heartbeat interval (2x heartbeat interval). This ensures
errors remain available for at least one retry cycle while preventing
unbounded growth. 2. Priority queue for efficient expiry: Uses a
min-heap to track entries by expiration time, enabling efficient cleanup
of expired entries during cache operations. 3. Capacity enforcement:
Limits cache size to prevent memory issues under high error rates. When
capacity is exceeded, oldest entries are evicted first. 4. Reference
equality checks: Uses eq for object identity comparison when cleaning up
stale entries, avoiding expensive value comparisons while correctly
handling entry updates.
Reviewers: Lucas Brutschy <lucasbru@apache.org>
This is the first part of cleaning up of the tests in `TaskManagerTest`
- Removed dead tests
- Added new tests as suggested earlier
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
Clarify rebalance callbacks behaviour (got some questions for
onPartitionsAssigned, docs where indeed confusing about the partitions
received in params). Reviewed all rebalance callbacks with it.
Reviewers: Bill Bejeck<bbejeck@apache.org>
In the consumer, we invoke the consumer rebalance onPartitionRevoked or
onPartitionLost callbacks, when the consumer closes. The point is that
the application may want to commit, or wipe the state if we are closing
unsuccessfully.
In the StreamsRebalanceListener, we did not implement this behavior,
which means when closing the consumer we may lose some progress, and in
the worst case also miss that we have to wipe our local state state
since we got fenced.
In this PR we implement StreamsRebalanceListenerInvoker, very similarly
to ConsumerRebalanceListenerInvoker and invoke it in Consumer.close.
Reviewers: Lianet Magrans <lmagrans@confluent.io>, Matthias J. Sax
<matthias@confluent.io>, TengYao Chi <frankvicky@apache.org>,
Uladzislau Blok <123193120+UladzislauBlok@users.noreply.github.com>
---------
Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
- Move the `RaftManager` interface to raft module, and remove the
`register` and `leaderAndEpoch` methods since they are already part of
the RaftClient APIs.
- Rename RaftManager.scala to KafkaRaftManager.scala.
Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi
<kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
Clarify timeout errors received on send if the case is topic not in
metadata vs partition not in metadata. Add integration tests showcases
the difference Follow-up from 4.1 fix for misleading timeout error
message (https://issues.apache.org/jira/browse/KAFKA-8862)
Reviewers: TengYao Chi <frankvicky@apache.org>, Kuan-Po Tseng
<brandboat@gmail.com>
This test case ensures that the parser can convert ISO8601 correctly.
However, when the local time falls on a different day than the UTC time,
there will be an off-by-one issue.
I changed the test to convert the local time and then compare it with
the expected local time. This should fix the off-by-one issue.
[Reference
link](https://github.com/apache/kafka/pull/18611#discussion_r2318146619)
Reviewers: Andrew Schofield <aschofield@confluent.io>
---------
Signed-off-by: Alex <wenhsuan.alexyu@gmail.com>
In the current solution, we only use a heap to select the right process,
but resort to linear search for selecting a member within a process.
This means use cases where a lot of threads run within the same process
can yield slow assignment. The number of threads in a process shouldn’t
scale arbitrarily (our assumed case for benchmarking of 50 threads in a
single process seems quite extreme already), however, we can optimize
for this case to reduce the runtime further.
Other assignment algorithms assign directly on the member-level, but we
cannot do this in Kafka Streams, since we cannot assign tasks to
processes that already own the task. Defining a heap directly on members
would mean that we may have to skip through 10s of member before finding
one that does not belong to a process that does not yet own the member.
Instead, we can define a separate heap for each process, which keeps the
members of the process by load. We can only keep the heap as long as we
are only changing the load of the top-most member (which we usually do).
This means we keep track of a lot of heaps, but since heaps are backed
by arrays in Java, this should not result in extreme memory
inefficiencies.
In our worst-performing benchmark, this improves the runtime by ~2x on
top of the optimization above.
Also piggybacked are some minor optimizations / clean-ups: -
initialize HashMaps and ArrayLists with the right capacity - fix some
comments - improve logging output
Note that this is a pure performance change, so there are no changes to
the unit tests.
Reviewers: Bill Bejeck<bbejeck@apache.org>
This PR migrates the `TransactionLogTest` from Scala to Java for better
consistency with the rest of the test suite and to simplify future
maintenance.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
The ApiError.fromThrowable(t) is going to return a generic
Errors.UNKNOWN_SERVER_ERROR back to the calling client (CLI for
instance) (eg if the broker has an authZ issue with ZK) and such
UnknownServerException should have a matching ERROR level log in the
broker logs IHMO to make it easier to troubleshoot
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
- Fix typo in `process.role`
- Fix formatting of quorum description commands
Reviewers: Lan Ding <isDing_L@163.com>, Ken Huang <s7133700@gmail.com>,
TengYao Chi <frankvicky@apache.org>
This pull request addresses KAFKA-19203 by replacing
`ApiError#exception` with `Error#exception` in `KafkaAdminClient`. The
previous use of `ApiError#exception` was redundant, as we only need the
exception without the additional wrapping of `ApiError`.
## Changes
- Replaced some usages of `ApiError#exception` with `Error#exception` in
`KafkaAdminClient`.
- Simplified exception handling logic to reduce unnecessary layers.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
The previous Javadoc example used the deprecated ClusterType. It is
now updated to use `types = {Type.KRAFT, Type.CO_KRAFT}`
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
## Changes
This PR improves the stability of the
PlaintextAdminIntegrationTest.testElectPreferredLeaders test by
introducing short Thread.sleep( ) delays before invoking:
- changePreferredLeader( )
- waitForBrokersOutOfIsr( )
## Reasons
- Metadata propagation for partition2 :
Kafka requires time to propagate the updated leader metadata across all
brokers. Without waiting, metadataCache may return outdated leader
information for partition2.
- Eviction of broker1 from the ISR :
To simulate a scenario where broker1 is no longer eligible as leader,
the test relies on broker1 being removed from the ISR (e.g., due to
intentional shutdown). This eviction is not instantaneous and requires a
brief delay before Kafka reflects the change.
Reviewers: PoAn Yang <payang@apache.org>, TengYao Chi
<kitingiao@gmail.com>, TaiJuWu <tjwu1217@gmail.com>, Ken Huang
<s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
This PR is a follow-up from https://github.com/apache/kafka/pull/20468.
Basically makes two things:
1. Moves the variable to the catch block as it is used only there.
2. Refactor FeaturesPublisher to handle exceptions the same as
ScramPublisher or other publishers :)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
---------
Signed-off-by: see-quick <maros.orsak159@gmail.com>
This improves the SmokeTestDriverIntegrationTest in three ways:
1) If a SmokeTestClient fails (enters a terminal ERROR state), the
SmokeTestDriverIntegrationTest currently times out, because it keeps
waiting for state NOT_RUNNING. This makes debugging quite difficult.
This minor change makes sure to just fail the test immediately, if a
SmokeTestClient enters the ERROR state.
2) If a test times out or fails prematurely, because a SmokeTestClient
crashed, the SmokeTestClients aren't shut down correctly, which will
affect the following test runs. Therefore, I am adding clean-up logic
that running SmokeTestClients in `@AfterAll`.
3) Finally, I found that the processingThread variation of this thread
triggers a subtle race condition. Since this features is currently not
actively developed, I disabled those variations and created a ticket to
reactivate the test.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Chia-Ping Tsai
<chia7712@gmail.com>, Bill Bejeck <bill@confluent.io>
---------
Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
Jira: [KAFKA-19676](https://issues.apache.org/jira/browse/KAFKA-19676)
All subclasses of EpochState do not throw an IOException when closing,
so catching it is unnecessary. We could override close to remove the
IOException declaration.
Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, TaiJuWu
<tjwu1217@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
That config has a `Validator` so we already automatically print the
valid values in the generated docs:
https://kafka.apache.org/documentation/#streamsconfigs_upgrade.from
That will be one less place to upgrade each time we make a new release.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
* The `ShareCoordinatorShard` maintains the the record offset
information for `SharePartitionKey`s in the
`ShareCoordinatorOffsetsManager` class.
* Replay of `ShareSnapshot`s in the shards are reflected in the offsets
manager including records created due to delete state.
* However, if the share partition delete is due to topic delete, no
record will ever be written for the same `SharePartitionKey` post the
delete tombstone (as topic id will not repeat).
As a result the offset manager will always consider the deleted share
partition's offset as the last redundant one.
* The fix is to make the offset manager aware of the tombstone records
and remove them from the redundant offset calculation.
* Unit tests have been updated for the same.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal
<apoorvmittal10@gmail.com>
In the original algorithm, standby tasks are assigned to a process that
previously owned the task only if it is “load-balanced”, meaning the
process has fewer tasks that members, or it is the least loaded
process. This is strong requirement, and will cause standby tasks to
often not get assigned to process that previously owned it.
Furthermore, the condition “is the least loaded process” is hard to
evaluate efficiently in this context.
We propose to instead use the same “below-quota” condition as in active
task assignment.
We compute a quota for active and standby tasks, by definiing numOfTasks
= numberOfActiveTasks+numberOfStandbyTasks and defining the quota as
numOfTasks/numberOfMembers rounded up. Whenever a member becomes “full”
(its assigned number of tasks is equal to numOfTasks) we deduct its
tasks from numOfTasks and decrement numberOfMembers and recompute the
quota, which means that the quota may be reduced by one during the
assignment process, to avoid uneven assignments.
A standby task can be assigned to a process that previously owned it,
whenever the process has fewer than numOfMembersOfProcess*quota.
This condition will, again, prioritize standby stickyness, and can be
evaluated in constant time.
In our worst-performing benchmark, this improves the runtime by 2.5x on
top of the previous optimizations, but 5x on the more important
incremental assignment case.
Reviewers: Bill Bejeck <bbejeck@apache.org>
The
[distutils](https://docs.python.org/3.13/whatsnew/3.12.html#distutils)
package is removed from Python 3.12.
Change `distutils` usage to `shutil`.
Reviewers: Mickael Maison <mimaison@apache.org>
---------
Signed-off-by: PoAn Yang <payang@apache.org>
Querying the oldest-open-iterator metric can result in a
NoSuchElementException when the last open iterator gets removed, due to
a race condition between the query and the metric update.
To avoid this race condition, this PR caches the metric result, to avoid
accessing the list of open iterator directly. We don't need to clear
this cache, because the entire metric is removed when the last iterator
gets removed.
Reviewers: Matthias J. Sax <matthias@confluent.io>
Similarly to what was done for
AsyncKafkaConsumerTest::testFailConstructor,
[here](https://github.com/apache/kafka/pull/20491)
Reviewers: Lianet Magrans <lmagrans@confluent.io>, Chia-Ping Tsai
<chia7712@gmail.com>
Add the explanation of `null` for DeleteAclsRequest#ResourceNameFilter
Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi
<kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
The method rollbackOrProcessStateUpdates in SharePartition received 2
separate lists of updatedStates (InFlightState) and stateBatches
(PersisterStateBatch). This PR introduces a new subclass called
`PersisterBatch` which encompasses both these objects.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
This is actually fixing a difference between the old and the new
assignor. Given the assignment ordering, the legacy assignor has a
preference for range-style assignments built in, that is, assigning
C1: 0_0, 1_0 C2: 0_1, 1_1
instead of
C1: 0_0, 0_1 C2: 1_0, 1_1
We add tests to both assignors to check for this behavior, and improve
the new assingor by enforcing corresponding orderings.
Reviewers: Bill Bejeck <bill@confluent.io>
Integration tests for Stream Admin related API
Previous PR: https://github.com/apache/kafka/pull/20244
This one adds:
- Integration test for Admin#listStreamsGroupOffsets API
- Integration test for Admin#deleteStreamsGroupOffsets API
- Integration test for Admin#alterStreamsGroupOffsets API
Reviewers: Alieh Saeedi <asaeedi@confluent.io>, Lucas Brutschy
<lucasbru@apache.org>
In `RaftUtil.addVoterResponse` and `RaftUtil.removeVoterResponse`
methods, when the input `errorMessage` is `null`, the returned string is
not actually null but `NONE`.
This introduces an inconsistency: semantically, `null` should represent
“no error message,” while `NONE` looks like a real string value and may
confuse clients.
Reviewers: Alyssa Huang <ahuang@confluent.io>, José Armando García
Sancio <jsancio@apache.org>, Anton Agestam <anton.agestam@aiven.io>,
Chia-Ping Tsai <chia7712@gmail.com>
The original name is confusing which could cause engineers to make a
mistake and confuse the `batchSize` with some other unit like number of
records.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
If there's a failure in the kafka consumer constructor, we attempt to
close it
2329def2ff/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java (L540)
In that case, it could be the case that some components may have not
been created, so we should consider some null checks to avoid noisy logs
about NPE.
This noisy logs have been reported with the console share consumer in a
similar scenario, so this task is to review and do a similar fix for the
Async if needed.
The fix is to check if handlers/invokers are null before trying to close
them. Similar to what was done here
https://github.com/apache/kafka/pull/20290
Reviewers: TengYao Chi <frankvicky@apache.org>, Lianet Magrans
<lmagrans@confluent.io>
The original test timeout when using new protocol, because it use
`ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG` as the exception's timeout,
which is 300s. Also the test for new protocol and old protocol use the
same group ID, so the failure will be hidden.
What I do:
1. Set the timeout as 5 secs so it can be captured within 10s
2. Use new appId for new protocol
Reviewers: Lucas Brutschy <lucasbru@apache.org>