Commit Graph

3295 Commits

Author SHA1 Message Date
Jhen-Yung Hsu 2e968560e0
MINOR: Cleanup simplify set initialization with Set.of (#19925)
Simplify Set initialization and reduce the overhead of creating extra
collections.

The changes mostly include:
- new HashSet<>(List.of(...))
- new HashSet<>(Arrays.asList(...)) / new HashSet<>(asList(...))
- new HashSet<>(Collections.singletonList()) / new
HashSet<>(singletonList())
- new HashSet<>(Collections.emptyList())
- new HashSet<>(Set.of())

This change takes the following into account, and we will not change to
Set.of in these scenarios:
- Require `mutability` (UnsupportedOperationException).
- Allow `duplicate` elements (IllegalArgumentException).
- Allow `null` elements (NullPointerException).
- Depend on `Ordering`. `Set.of` does not guarantee order, so it could
make tests flaky or break public interfaces.

Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
 <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2025-06-11 18:36:14 +08:00
Mickael Maison b704280a0a
MINOR: Bump trunk to 4.2.0-SNAPSHOT (#19944)
CI / build (push) Waiting to run Details
Update the version as per
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=34840886#ReleaseProcess-CutBranches

Reviewers: Luke Chen <showuon@gmail.com>
2025-06-11 09:24:57 +02:00
Lucas Brutschy 793dcee541
KAFKA-19376: Throw an error message if any unsupported feature is used with KIP-1071 (#19908)
We should be mindful of ours users and let them know early if they are
using an unsupported feature in 4.1.

Unsupported features:

- Regular expressions
- Warm-up replicas (high availability assignor)
- Static membership
- Standby replicas enabled through local config
- Named topologies (already checked)
- Non-default kafka-client supplier

Reviewers: Bill Bejeck <bbejeck@apache.org>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-10 11:13:34 +02:00
Matthias J. Sax 0adc6fa3e1
KAFKA-19271: allow intercepting internal method call (#19832)
CI / build (push) Waiting to run Details
To allow intercepting the internal subscribe call to the async-consumer,
we need to extend ConsumerWrapper interface accordingly, instead of
returning the wrapped async-consumer back to the KS runtime.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2025-06-09 07:28:13 -07:00
Janindu Pathirana 4d6cf3efef
KAFKA-18913: Start state updater in task manager (#19889)
Updated the code to start the State Updater Thread only after the Stream
Thread is started.

Changes done :
1. Moved the starting of the StateUpdater thread to a new init method in
the TaskManager.
2. Called the init of TaskManager in the run method of the StreamThread.
3. Updated the test cases in the StreamThreadTest to mimic the
aforementioned behaviour.

Reviewers: Bruno Cadonna <cadonna@apache.org>
2025-06-06 11:14:41 +02:00
Loïc GREFFIER 3edb406f98
KAFKA-16505: Add source raw key and value (#18739)
This PR is part of the KIP-1034.

It brings the support for the source raw key and the source raw
value in the `ErrorHandlerContext`. Required by the routing to DLQ implemented
by https://github.com/apache/kafka/pull/17942.

Reviewers: Bruno Cadonna <cadonna@apache.org>

Co-authored-by: Damien Gasparina <d.gasparina@gmail.com>
2025-06-05 10:35:03 +02:00
lorcan ef14f76fb3
KAFKA-6629: parameterise SegmentedCacheFunctionTest for session key schemas (#19404)
Addresses:
[KAFKA-6629](https://issues.apache.org/jira/browse/KAFKA-6629)

Adds configuration for the SessionKeySchema and parameterises the
existing tests  so that both WindowKeys and SessionKeys are tested under
the existing unit tests.

Reviewers: Bill Bejeck <bbejeck@apache.org>

---------

Co-authored-by: Lorcan <lorcanjames1@gmail.com>
2025-05-29 11:57:10 -04:00
Ken Huang bcda92b5b9
KAFKA-19080 The constraint on segment.ms is not enforced at topic level (#19371)
CI / build (push) Waiting to run Details
The main issue was that we forgot to set
`TopicConfig.SEGMENT_BYTES_CONFIG` to at least `1024 * 1024`, which
caused problems in tests with small segment sizes.

To address this, we introduced a new internal config:
`LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG`, allowing us to set smaller
segment bytes specifically for testing purposes.

We also updated the logic so that if a user configures the topic-level
segment bytes without explicitly setting the internal config, the
internal value will no longer be returned to the user.

In addition, we removed
`MetadataLogConfig#METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG` and added
three new internal configurations:
- `INTERNAL_MAX_BATCH_SIZE_IN_BYTES_CONFIG`
- `INTERNAL_MAX_FETCH_SIZE_IN_BYTES_CONFIG`
- `INTERNAL_DELETE_DELAY_MILLIS_CONFIG`

Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-05-25 20:57:22 +08:00
PoAn Yang 30d7c71f09
KAFKA-18904: Add Admin#listConfigResources [2/N] (#19743)
CI / build (push) Waiting to run Details
* Add new functions `listConfigResources(Set<ConfigResource.Type>
configResourceTypes, ListConfigResourcesOptions options)` and
`listConfigResources()` to `Admin` interface.
  * New functions can list all kind of config resource types.
  * If input is a set with a type other than `CLIENT_METRICS` and
request version is 0, return `UnsupportedVersionException`.
* Deprecate functions
`listClientMetricsResources(ListClientMetricsResourcesOptions options)`
and `listClientMetricsResources()`.
* Deprecate classes `ListClientMetricsResourcesResult` and
`ClientMetricsResourceListing`.
* Change `ClientMetricsCommand` to use `listConfigResources`.
* Add integration tests to `PlaintextAdminIntegrationTest.java`.
* Add unit tests to `KafkaAdminClientTest.java`.

Reviewers: Andrew Schofield <aschofield@confluent.io>

---------

Signed-off-by: PoAn Yang <payang@apache.org>
2025-05-22 16:05:35 +01:00
lorcan e88c10d595
KAFKA-4650: Add unit tests for GraphNode class (#18951)
CI / build (push) Waiting to run Details
Unit tests added for the class GraphNode.

Change applied to GraphNode parentNodes() function to return a copy of
the collection, which is consistent with how the children() collection
is returned.

Reviewers: Bill Bejeck <bbejeck@apache.org>

---------

Co-authored-by: Lorcan <lorcanjames1@gmail.com>
2025-05-21 08:58:24 -04:00
Lucas Brutschy f26974b16d
KAFKA-19202: Enable KIP-1071 in streams_eos_test (#19700)
CI / build (push) Waiting to run Details
Enable next system test with KIP-1071.

Some of the validation inside the test did not make sense for KIP-1071.
This is because in KIP-1071, if a member leaves or joins the group, not
all members may enter a REBALANCING state. We use the wrapper introduced
in   [KAFKA-19271](https://issues.apache.org/jira/browse/KAFKA-19271)
to print a log line whenever the member epoch is bumped, which is  the
only way a member can "indirectly" observe that other members  are
rebalancing.

Reviewers: Bill Bejeck <bill@confluent.io>
2025-05-17 21:20:47 +02:00
Matthias J. Sax 34e426b517
KAFKA-19271: Add internal ConsumerWrapper (#19697)
With KIP-1071 enabled, the main consumer is created differently,  side
stepping `KafkaClientSupplier`.

To allow injection test wrappers, we add an internal ConsumerWrapper,
until we define a new public interface.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2025-05-16 11:57:37 +02:00
Matthias J. Sax 8fc41f9ca2
KAFKA-19171: Kafka Streams crashes with UnsupportedOperationException (#19507)
This PR fixes a regression bug introduced with KAFKA-17203. We need to
pass in mutable collections into `closeTaskClean(...)`.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Bruno Cadonna <bruno@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>
2025-05-15 21:37:04 -07:00
Matthias J. Sax bbf48ad677
KAFKA-19208: KStream-GlobalKTable join should not drop left-null-key record (#19580)
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2025-05-15 18:34:07 -07:00
Kuan-Po Tseng cafe83f928
KAFKA-19275 client-state and thread-state metrics are always "Unavailable" (#19712)
Fix the issue where JMC is unable to correctly display client-state and
thread-state metrics. The root cause is that these two metrics directly
return the `State` class to JMX. If the user has not set up the RMI
server, JMC or other monitoring tools will be unable to interpret the
`State` class. To resolve this, we should return a string representation
of the state instead of the State class in these two metrics.

Reviewers: Luke Chen <showuon@gmail.com>, Ken Huang
 <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-14 14:07:32 +08:00
Matthias J. Sax 0b81d6c780
MINOR: avoid double brace initialization (#19667)
Reviewers: Bill Bejeck <bill@confluent.io>
2025-05-09 11:52:01 -07:00
Lucas Brutschy 3f465fc1b6
KAFKA-19202: Enable KIP-1071 in streams_standby_replica_test.py (#19625)
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>
2025-05-07 09:43:11 +02:00
Matthias J. Sax c8005a543e
MINOR: improve AdjustStreamThreadCountTest (#19617)
CI / build (push) Waiting to run Details
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>
2025-05-05 13:23:50 -07:00
Matthias J. Sax bc7e57242d
MINOR: remove @FunctionalInterface annotation (#19618)
CI / build (push) Waiting to run Details
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>
2025-05-02 18:02:38 +08:00
Matthias J. Sax 44d2741b4c
MINOR: fix bug in TimeWindowedDeserializerTest (#19570)
CI / build (push) Waiting to run Details
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>
2025-04-30 17:58:22 -07:00
lorcan 009fc7cdbf
MINOR: Move candidateClients set creation outside of task loop for StickyTaskAssignor (#19511)
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>
2025-04-30 10:22:04 -07:00
Lucas Brutschy 53afb1e0c5
KAFKA-19202: Enable KIP-1071 in streams_relational_smoke_test (#19604)
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>
2025-04-30 18:41:37 +02:00
TengYao Chi 08f6042f7a
HOTFIX: Fix IQv2EndpointToPartitionsIntegrationTest (#19597)
CI / build (push) Waiting to run Details
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>
2025-04-30 15:33:09 +08:00
Matthias J. Sax b0a26bc2f4
KAFKA-19173: Add `Feature` for "streams" group (#19509)
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>
2025-04-29 22:51:10 -07:00
Bill Bejeck 431cffc93f
KAFKA-19135 Migrate initial IQ support for KIP-1071 from feature branch to trunk (#19588)
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>
2025-04-29 20:08:49 -04:00
Dmitry Werner 988fa3f272
MINOR: Small cleanups in streams tests (#19446)
- Fixed typos
- Fixed IDEA code inspection warnings
- Removed unused fields and methods

Reviewers: PoAn Yang <payang@apache.org>, TengYao Chi <frankvicky@apache.org>, Matthias J. Sax <matthias@confluent.io>
2025-04-29 16:26:50 -07:00
Matthias J. Sax 0cf2f0e55d
MINOR: update KafkaStreamsTelemetryIntegrationTest for Java17 (#19567)
Reviewers: Bill Bejeck <bill@confluent.io>
2025-04-25 16:43:18 -07:00
Chirag Wadhwa 2f9c2dd828
KAFKA-16718-3/n: Added the ShareGroupStatePartitionMetadata record during deletion of share group offsets (#19478)
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>
2025-04-25 22:01:48 +01:00
Lucas Brutschy d087ade527
HOTFIX: Fix StreamThreadTest (#19562)
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>
2025-04-25 15:03:39 +02:00
Lucas Brutschy 732ed0696b
KAFKA-19190: Handle shutdown application correctly (#19544)
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>
2025-04-25 09:56:09 +02:00
Alieh Saeedi ee4debb9f0
KAFKA-19128: Kafka Streams should not get offsets when close dirty (#19450)
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>
2025-04-24 21:23:07 -07:00
PoAn Yang 3fae785ea0
KAFKA-19110: Add missing unit test for Streams-consumer integration (#19457)
- 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>
2025-04-24 10:38:22 +02:00
Andrew Schofield 8d66481a83
KAFKA-17897 Deprecate Admin.listConsumerGroups (#19477)
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>
2025-04-17 23:00:57 +08:00
Hong-Yi Chen 144101a7c1
KAFKA-19054: StreamThread exception handling with SHUTDOWN_APPLICATION may trigger a tight loop with MANY logs (#19394)
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>
2025-04-16 20:35:43 -07:00
Liam Miller-Cushon 1bb0c9a54b
MINOR: Fix an incorrect assignment in StreamsUncaughtExceptionHandlerIntegrationTest (#18837)
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>
2025-04-16 20:25:07 -07:00
Chris Flood 67fa3656cc
MINOR: Fix Streams Position thread-safety (#19480)
* 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>
2025-04-16 14:45:38 -05:00
xijiu 8bdd73c803
KAFKA-19137 Use `StandardCharsets.UTF_8` instead of `StandardCharsets.UTF_8.name()` (#19464)
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>
2025-04-15 23:44:00 +08:00
Lucas Brutschy 9e620287f1
KAFKA-19118: Enable KIP-1071 in StandbyTaskCreationIntegrationTest (#19438)
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>
2025-04-11 11:20:54 +02:00
Bruno Cadonna c11938c926
KAFKA-19124: Use consumer background event queue for Streams events (#19421)
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>
2025-04-10 21:06:06 +02:00
Lucas Brutschy c65a161cd8
KAFKA-19118: Enable KIP-1071 in InternalTopicIntegrationTest (#19425)
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>
2025-04-10 16:57:38 +02:00
Florian Hussonnois eeb1214ba8
KAFKA-18962: Fix onBatchRestored call in GlobalStateManagerImpl (#19188)
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>
2025-04-09 13:17:38 -07:00
Bruno Cadonna 2a370ed721
KAFKA-19037: Integrate consumer-side code with Streams (#19377)
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>
2025-04-09 13:26:51 +02:00
Chirag Wadhwa 5148174196
KAFKA-16718-2/n: KafkaAdminClient and GroupCoordinator implementation for DeleteShareGroupOffsets RPC (#18976)
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>
2025-04-09 07:31:06 +01:00
Ayoub Omari b963e58000
KAFKA-16407: Fix foreign key INNER join on change of FK from/to a null value (#19303)
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>
2025-04-05 20:13:31 -07:00
PoAn Yang d6ff7b80dd
KAFKA-13747: refactor TopologyTest to test different store type parametrized (#19323)
Use `ParameterizedTest` to test both `IN_MEMORY` and `ROCKS_DB` store
type in `TopologyTest`.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2025-04-04 14:45:13 -07:00
lorcan 6af849f864
KAFKA-7952: use in memory stores for KTable test (#19218)
Switch to in-memory story for KTable-KTable left-join test.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2025-04-04 09:22:08 -07:00
nilmadhab mondal d35ab4d27a
KAFKA-18713: Fix FK Left-Join result race condition (#19005)
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>
2025-04-03 16:22:47 -07:00
TengYao Chi 5eb4e116bc
KAFKA-18067: Add a flag to disable producer reset during active task creator shutting down (#19269)
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>
2025-04-03 14:13:22 -07:00
Thomas Gebert db4e74b46e
MINOR: Add Functional Interface annotation to interfaces used by Lambdas (#19234)
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>
2025-04-03 09:30:56 -07:00
Ken Huang d619cc5b88
KAFKA-19058 Running the streams/streams-scala module tests produces a streams-scala.log (#19324)
Remove streams-scala module log4j2 file appender.

Reviewers: PoAn Yang <payang@apache.org>, Matthias J. Sax <matthias@confluent.io>
2025-04-02 12:12:16 -07:00