New system test for KIP-1071.
Standby replicas need to be enabled via `kafka-configs.sh`.
Reviewers: Bill Bejeck <bill@confluent.io>, Matthias J. Sax
<matthias@confluent.io>
The test is failing once in a while but there is not enough information
in the logs to determine the root cause. Adding more information, and
fixing thread resource leak.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
BatchingStateRestoreCallback's default implemeantion of restore() lead
to waraning `FunctionalInterfaceMethodChanged`.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, PoAn Yang
<payang@apache.org>, Ken Huang <s7133700@gmail.com>, TengYao Chi
<frankvicky@apache.org>
Test throws `NumberFormatException` and thus still passed as this
exception extends `IllegalArgumentException`. However, the test does
not verify what it is supposed to verify.
Piggybacking some code cleanup to all related files.
Reviewers: PoAn Yang <payang@apache.org>, Ken Huang <s7133700@gmail.com>, Bill Bejeck <bill@confluent.io>
This PR moves the computation of the "client list", which is the same
for all tasks, out of the loop, to avoid unnecessary re-computation.
Reviewers: Matthias J. Sax <matthias@confluent.io>
Enable next system test with KIP-1071.
Also fixes the other KIP-1071 system tests, which now require enabling
the unstable `streams.version` feature.
Reviewers: Bill Bejeck <bbejeck@apache.org>
It seems that IQv2EndpointToPartitionsIntegrationTest uses a
non-existent method to create `EmbeddedKafkaCluster`
Reviewers: Luke Chen <showuon@gmail.com>, PoAn Yang <payang@apache.org>,
Ken Huang <s7133700@gmail.com>
Add new StreamsGroupFeature, disabled by default, and add "streams" as
default value to `group.coordinator.rebalance.protocols`.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Jacot
<david.jacot@gmail.com>, Lucas Brutschy <lbrutschy@confluent.io>,
Justine Olshan <jolshan@confluent.io>, Andrew Schofield
<aschofield@confluent.io>, Jun Rao <jun@confluent.io>
This PR is a migration of the initial IQ support for KIP-1071 from the
feature branch to trunk. It includes a parameterized integration test
that expects the same results whether using either the classic or new
streams group protocol.
Note that this PR will deliver IQ information in each heartbeat
response. A follow-up PR will change that to be only sending IQ
information when assignments change.
Reviewers Lucas Brutschy <lucasbru@apache.org>
This is a follow up PR for implementation of DeleteShareGroupOffsets
RPC. This PR adds the ShareGroupStatePartitionMetadata record to
__consumer__offsets topic to make sure the topic is removed from the
initializedTopics list. This PR also removes partitions from the request
and response schemas for DeleteShareGroupState RPC
Reviewers: Sushant Mahajan <smahajan@confluent.io>, Andrew Schofield <aschofield@confluent.io>
Commit 732ed06 changed the logic of handling shutdowns, but in parallel
commit 3fae785 had introduced a new unit test for checking how to shut
down, which was broken by the later commit.
Reviewers: David Jacot <djacot@confluent.io>
If the streams rebalance protocol is enabled in
StreamsUncaughtExceptionHandlerIntegrationTest, the streams application
does not shut down correctly upon error.
There are two causes for this. First, sometimes, the SHUTDOWN_APPLICATION
code only sent with the leave heartbeat, but that is not handled broker
side. Second, the SHUTDOWN_APPLICATION code wasn't properly handled
client-side at all.
Reviewers: Bruno Cadonna <cadonna@apache.org>, Bill Bejeck
<bill@confluent.io>, PoAn Yang <payang@apache.org>
Kafka Streams calls `prepareCommit()` in `Taskmanager#closeTaskDirty()`.
However, the dirty task must not get committed and therefore,
prepare-commit tasks such as getting offsets should not be needed as
well. The only thing needed before closing a task dirty is flushing.
Therefore, separating `flush` and `prepareCommit` could be a good fix.
Reviewers: Bill Bejeck <bill@confluent.io>, Matthias J. Sax <matthias@confluent.io>
- Construct `AsyncKafkaConsumer` constructor and verify that the
`RequestManagers.supplier()` contains Streams-specific data structures.
- Verify that `RequestManagers` constructs the Streams request managers
correctly
- Test `StreamsGroupHeartbeatManager#resetPollTimer()`
- Test `StreamsOnTasksRevokedCallbackCompletedEvent`,
`StreamsOnTasksAssignedCallbackCompletedEvent`, and
`StreamsOnAllTasksLostCallbackCompletedEvent` in
`ApplicationEventProcessor`
- Test `DefaultStreamsRebalanceListener`
- Test `StreamThread`.
- Test `handleStreamsRebalanceData`.
- Test `StreamsRebalanceData`.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Bill Bejeck <bill@confluent.io>
Signed-off-by: PoAn Yang <payang@apache.org>
The final part of KIP-1043 is to deprecate Admin.listConsumerGroups() in
favour of Admin.listGroups() which works for all group types.
Reviewers: PoAn Yang <payang@apache.org>, Chia-Ping Tsai
<chia7712@gmail.com>
Under the `SHUTDOWN_APPLICATION` configuration in Kafka Streams, a tight
loop in the shutdown process can flood logs with repeated messages. This
PR introduces a check to ensure that the shutdown log is emitted only
once every 10 seconds, thereby preventing log flooding.
Reviewers: PoAn Yang <payang@apache.org>, Matthias J. Sax <matthias@confluent.io>
This constructor was not initializing a field with the constructor
argument, the extra `} {` was ending the constructor body and creating
an instance initializer block that assigned the field to itself.
Reviewers: Matthias J. Sax <matthias@confluent.io>
* Fixes a thread-safety bug in the Kafka Streams Position class
* Adds a multithreaded test to validate the fix and prevent regressions
Reviewers: John Roesler <vvcephei@apache.org>
Replace `StandardCharsets.UTF_8.name()` with `StandardCharsets.UTF_8` to
avoid UnsupportedEncodingException and optimize the related code at the
same time.
Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
<payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
Enable KIP-1071 parameter in `StandbyTaskCreationIntegrationTest`.
Required a fix: In `ChangelogTopic.setup`, we actually need to return
both the source-topic (optimized) and the non-source-topic changelog
topics, since otherwise we will not find the partition number later on.
Extended `EmbeddedKafkaCluster` to set the number of standby replicas
dynamically for the group. We need to initialize it to one for the
integration test to go through.
Reviewers: Bill Bejeck <bbejeck@apache.org>
In the first version of the integration of the stream thread with the
new Streams rebalance protocol, the consumer used a dedicated event
queue for Streams/specific background events to request the stream
thread to call the rebalance callbacks. That led to an issue where the
consumer times out when unsubscribing.
This commit gets rid of the dedicated queue and incorporates the
Streams-specific background events into event queue used by the
consumer.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
KIP-1071 creates internal topics broker-side, so this test checks
whether, when KIP-1071 is enabled, basically the same topics are
created.
It also adds a little helper method in `EmbeddedKafkaCluster`, so that
fewer code changes are required to enable KIP-1071. We use that helper
in the already enabled SmokeTestDriverIntegrationTest and revert some of
the changes there (making the cluster `final` again).
Reviewers: Bill Bejeck <bbejeck@apache.org>, PoAn Yang
<payang@apache.org>
Call the StateRestoreListener#onBatchRestored with numRestored and not
the totalRestored when reprocessing state
See: https://issues.apache.org/jira/browse/KAFKA-18962
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Matthias
Sax <mjsax@apache.org>
The consumer adaptations for the new Streams rebalance protocol need to
be integrated into the Streams code. This commit does the following:
- creates an async Kafka consumer
- with a Streams heartbeat request manager
- with a Streams membership manager
- integrates consumer code with the Streams membership manager and the
Streams heartbeat request manager
- processes the events from the consumer network thread (a.k.a.
background thread)
that request the invocation of the "on tasks revoked", "on tasks
assigned", and "on all tasks lost"
callbacks
- executes the callbacks
- sends to the consumer network thread the events signalling the
execution of the callbacks
- adapts SmokeTestDriverIntegrationTest to use the new Streams rebalance
protocol
This commit misses some unit test coverage, but it also unblocks other
work on trunk regarding the new Streams rebalance protocol. The missing
unit tests will be added soon.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
This PR contains the implementation of KafkaAdminClient and
GroupCoordinator for DeleteShareGroupOffsets RPC.
- Added `deleteShareGroupOffsets` to `KafkaAdminClient`
- Added implementation for `handleDeleteShareGroupOffsetsRequest` in
`KafkaApis.scala`
- Added `deleteShareGroupOffsets` to `GroupCoordinator` as well.
internally this makes use of `persister.deleteState` to persist the
changes in share coordinator
Reviewers: Andrew Schofield <aschofield@confluent.io>, Sushant Mahajan <smahajan@confluent.io>
Fixes both KAFKA-16407 and KAFKA-16434.
Summary of existing issues:
- We are ignoring new left record when its previous FK value is null
- We do not unset foreign key join result when FK becomes null
Reviewers: Matthias J. Sax <matthias@confluent.io>
When a row in a FK-join left table is updated, we should send a "delete
subscription with no response" for the old FK to the right hand side, to
avoid getting two responses from the right hand side. Only the "new
subscription" for the new FK should request a response. If two responses
are requested, there is a race condition for which both responses could
be processed in the wrong order, leading to an incorrect join result.
This PR fixes the "delete subscription" case accordingly, to no request
a response.
Reviewers: Matthias J. Sax <matthias@confluent.io>
JIRA: KAFKA-18067
Fix producer client double-closing issue in Kafka Streams.
During StreamThread shutdown, TaskManager closes first, which closes the
producer client. Later, calling `unsubscribe` on the main consumer may
trigger the `onPartitionsLost` callback, attempting to reset
StreamsProducer when EOS is enabled. This causes an already closed
producer to be closed twice while the newly created producer is never
closed.
In detail:
This patch adds a flag to control the producer reset and has a new
method to change this flag, which is only invoked in
`ActiveTaskCreator#close`.
This would guarantee that the disable reset producer will only occur
when StreamThread shuts down.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Matthias Sax <mjsax@apache.org>
Adds the FunctionalInterface annotation to relevant Kafka Streams
classes. While this is not strictly required for Java, it's still best
practice and also useful for better integration with other JVM
languages, for example Clojure, to allow using these interfaces as
lambdas.
Reviewers: Matthias J. Sax <matthias@confluent.io>
This patch is part of KIP-939 [Support Participation in
2PC](https://cwiki.apache.org/confluence/display/KAFKA/KIP-939%3A+Support+Participation+in+2PC)
The kafka-transactions.sh tool will support a new command
--forceTerminateTransaction It has one required argument
--transactionalId that would take the transactional id for the
transaction to be terminated.
The command uses the existing Admin#fenceProducers method to forcefully
abort the transaction associated with the specified transactional ID.
Under the hood, it sends an InitProducerId request to the transaction
coordinator with the given transactional ID and keepPreparedTxn = false
by default. This is aligned with the functionality outlined in the KIP.
We will be creating a new public method in the Admin Client **public
TerminateTransactionResult forceTerminateTransaction(String
transactionalId)**, and re-use the existing fence producer method.
Reviewers: Artem Livshits <alivshits@confluent.io>, Justine Olshan <jolshan@confluent.io>
When implementing
[KIP-1091](https://cwiki.apache.org/confluence/display/KAFKA/KIP-1091%3A+Improved+Kafka+Streams+operator+metrics)
metrics for the Global Stream thread was overlooked. This ticket adds
the Global Thread metrics so they are available via the KIP-1076 process
of adding external Kafka metrics.
The existing integration test has been updated to confirm GlobalThread
metrics are sent via the broker plugin.
Reviewers: Matthias Sax <mjsax@apache.org>
The `window.size.ms` and `window.inner.class.serde` are not a true
KafkaStreams config, and are ignored when set from a KStreams
application. Both belong on the client.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
Signed-off-by: PoAn Yang <payang@apache.org>
The generated request data type's constructors take Readable as an input. However, the parse method in the
AbstractRequest takes a ByteBuffer as input. So to create the corresponding request data objects, each individual
concrete Request classes wraps the ByteBuffer into a ByteBufferAccessor.
This is boilerplate code present in all the concrete request classes. This changes AbstractRequest's parse method so that subclasses can simply pass the `Readable` they get directly to request data classes.
The same change is made to the serialize method to maintain symmetry.
Reviewers: Ismael Juma <ismael@juma.me.uk>, José Armando García Sancio
<jsancio@apache.org>, Artem Livshits <alivshits@confluent.io>,
Truc Nguyen <trnguyen@confluent.io>
Fixes two issues:
- only commit TX if no revoked tasks need to be committed
- commit revoked tasks after punctuation triggered
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Anna Sophie Blee-Goldman <sophie@responsive.dev>, Bruno Cadonna <bruno@confluent.io>, Bill Bejeck <bill@confluent.io>
Exceptions that are caught during closing a task dirty are swallowed.
The corresponding log messages are on error level which is misleading
since the exceptions do not cause any special handling or crash.
This commit downgrades the log messages of swallowed exceptions to
warnings and explains in the log messages that the exceptions are
swallowed.
Reviewers: Bill Bejeck <bill@confluent.io>
Recently, we found a regression that could have been detected by static
analysis, since a local variable wasn't being passed to a method during
a refactoring, and was left unused. It was fixed in
[7a749b5](7a749b589f),
but almost slipped into 4.0. Unused variables are typically detected by
IDEs, but this is insufficient to prevent these kinds of bugs. This
change enables unused local variable detection in checkstyle for Kafka.
A few notes on the usage:
- There are two situations in which people actually want to have a local
variable but not use it. First, there are `for (Type ignored:
collection)` loops which have to loop `collection.length` number of
times, but that do not use `ignored` in the loop body. These are
typically still easier to read than a classical `for` loop. Second, some
IDEs detect it if a return value of a function such as `File.delete` is
not being used. In this case, people sometimes store the result in an
unused local variable to make ignoring the return value explicit and to
avoid the squiggly lines.
- In Java 22, unsued local variables can be omitted by using a single
underscore `_`. This is supported by checkstyle. In pre-22 versions,
IntelliJ allows such variables to be named `ignored` to suppress the
unused local variable warning. This pattern is often (but not
consistently) used in the Kafka codebase. This is, however, not
supported by checkstyle.
Since we cannot switch to Java 22, yet, and we want to use automated
detection using checkstyle, we have to resort to prefixing the unused
local variables with `@SuppressWarnings("UnusedLocalVariable")`. We have
to apply this in 11 cases across the Kafka codebase. While not being
pretty, I'd argue it's worth it to prevent bugs like the one fixed in
[7a749b5](7a749b589f).
Reviewers: Andrew Schofield <aschofield@confluent.io>, David Arthur
<mumrah@gmail.com>, Matthias J. Sax <matthias@confluent.io>, Bruno
Cadonna <cadonna@apache.org>, Kirk True <ktrue@confluent.io>
Adds `describeStreamsGroup` to Admin API.
This exposes the result of the `DESCRIBE_STREAMS_GROUP` RPC in the Admin
API.
Reviewers: Bill Bejeck <bill@confluent.io>
Implement Admin API extensions beyond list/describe group (delete group,
offset-related APIs).
* adds methods for describing and manipulating offsets, as described in
KIP-1071
* adds corresponding unit tests
These are doing the exact same thing as the corresponding consumer group
counter-parts.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
Implement Admin API extensions beyond list/describe group (delete group, offset-related APIs).
* adds methods for describing and manipulating offsets, as described in KIP-1071
* adds corresponding unit tests
These are doing the exact same thing as the corresponding consumer group counter-parts.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>