Commit Graph

3188 Commits

Author SHA1 Message Date
Matthias J. Sax 923086dba2 KAFKA-19171: Kafka Streams crashes with UnsupportedOperationException (#19507)
CI / build (push) Has been cancelled Details
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:40:08 -07:00
Matthias J. Sax 62c6697ac9 KAFKA-19208: KStream-GlobalKTable join should not drop left-null-key record (#19580)
CI / build (push) Waiting to run Details
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2025-05-15 18:37:39 -07:00
Kuan-Po Tseng f99db0804e KAFKA-19275 client-state and thread-state metrics are always "Unavailable" (#19712)
CI / build (push) Has been cancelled Details
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:08:01 +08:00
Hong-Yi Chen 8a515da2c8 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:46:17 -07:00
Florian Hussonnois de27409e30 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:38:03 -07:00
Ayoub Omari 71c9d83b20 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:46:56 -07:00
nilmadhab mondal 2c48809fad 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:53:01 -07:00
TengYao Chi b0b4f42f4c 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:14:26 -07:00
David Jacot 16da5a885e
MINOR: Bump to 4.0.1-SNAPSHOT (#19224)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-18 11:56:02 +01:00
Matthias J. Sax 3e791d7d48 KAFKA-18943: Kafka Streams incorrectly commits TX during task revokation (#19164)
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>
2025-03-13 16:56:31 -07:00
Lucas Brutschy 9ef4d5ab86 KAFKA-18917: TransformValues throws NPE (#19089)
When `transformValues` is used with a `Materialized` instance, but
without a queryable name, a `NullPointerException` is thrown. To
preserve the semantics present in 3.9, we need to avoid materialization
when a queryable name is not present.

Reviewers: Bruno Cadonna <cadonna@apache.org>
2025-03-04 17:51:46 +01:00
Matthias J. Sax dadd02de8b
HOTFIX: remove PageView example to support Java11 for :streams:examples module (#19052)
The PageView example depends on Connect to pull in Json (de)serializers,
but Connect does not support Java11 any longer.

To allow supporting Java11 for the Kafka Streams examples, this PR
removes the PageView examples and Connect dependency.

Reviewers: Bruno Cadonna <bruno@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <djacot@confluent.io>
2025-03-03 10:00:15 -08:00
Bruno Cadonna d91fbc7fa7 HOTFIX: Revert "KAFKA-18067: Kafka Streams can leak Producer client under EOS (#17931)" (#19078)
This reverts commit e8837465a5.

The commit that is reverted prevents Kafka Streams from re-initializing
its transactional producer. If an exception that fences the
transactional producer occurs, the producer is not re-initialized during
the handling of the exception. That causes an infinite loop of
ProducerFencedExceptions with corresponding rebalances.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, David Jacot
<djacot@confluent.io>
2025-03-03 13:39:59 +01:00
A. Sophie Blee-Goldman ed23f6be90 KAFKA-18839: Drop EAGER rebalancing support in Kafka Streams (#18988)
In 3.1 we deprecated the eager rebalancing protocol and marked it for
removal in a later release. We aim to officially drop support and remove
the protocol from Streams in 4.0.

The effect of this PR is that it will no longer be possible to perform a
live upgrade Kafka Streams directly to 4.0 from version 2.3 or below.
Users will have to go through a bridge release between 2.4 - 3.9
instead.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2025-02-25 19:08:51 -08:00
Matthias J. Sax 7a749b589f HOTFIX: StoreChangelogReader should require stable consumer group (#18901)
Fixing regression bug, introduced by beac86f049

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Bruno Cadonna <bruno@confluent.io>
2025-02-17 12:56:50 -08:00
Bill Bejeck 91958fce6a MINOR: Adjust javadoc to reflect the correct status of standby task TopicPartition (#18892)
KIP-744 introduced the StreamsMetadata class as part of the implementation. In the KIP, the javadoc for the standbyTopicPartitions states that the method returns the set of source TopicPartition that it represents as a standby. The current javadoc states that it represents the changelog TopicPartition(s). While the partitions of the source and changelog topics will match, the javadoc needs to be updated to reflect the correct behavior.

Note that the deprecated o.a.k.streams.state.StreamsMetadata#standbyTopicPartitions method also describes the set of TopicPartition being source TopicPartition.

Reviewers: Matthias Sax<mjsax@apache.org>
2025-02-13 14:11:02 -05:00
Swikar Patel 44a0dce3de KAFKA-15443: Upgrade RocksDB to 9.7.3 (#18275)
This PR upgrades RocksDB from 7.9.2 to 9.7.3 and addresses the following compatibility issues introduced by the RocksDB upgrade:

- Removal of AccessHint: The AccessHint class was completely removed in RocksDB 9.7.3. This required removing all import statements, variable declarations, method parameters, method return types, and static method calls related to AccessHint in RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter.java RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest.java Unused methods are removed in RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter.java
- Removal of NO_FILE_CLOSES: The NO_FILE_CLOSES metric was also removed in RocksDB 9.7.3. The calculation for numberOfOpenFiles in RocksDBMetricsRecorder.java has been adjusted to now track the total number of file opens since the last reset. The previous calculation, which subtracted NO_FILE_CLOSES from NO_FILE_OPENS, is no longer possible. The reason RocksDB removed NO_FILE_CLOSES seems to be that it did not properly work: https://github.com/search?q=repo%3Afacebook%2Frocksdb+NO_FILE_CLOSES&type=issues
- Removal of methods related to compressed block cache configuration in BlockBasedTableConfig
- Change of the signature of org.rocksdb.Options.setLogger()

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Matthias J. Sax <matthias@confluent.io>, Bruno Cadonna <cadonna@apache.org>
2025-02-12 10:26:25 +01:00
Matthias J. Sax 562ecf4a83 MINOR: make leaking public member in StreamsConfig non-private
KIP-1112 specified PROCESSOR_WRAPPER_CLASS_DOC as `private` and it should not be public.
We need to make to package-private though, to allow TopologyConfig to use it.
2025-02-07 21:23:10 -08:00
Lucas Brutschy 54eadda130
MINOR: Fix streams smoke test flush records (#18830)
In the streams smoke test, flush records that are appended to the input topics, to advance the stream time so that all suppressed windows are flushed at the end of the test. The records are created with record time equal to current time + 2 days. caf0b67 changed the broker defaults so that records more than one hour in the future are rejected by the broker. This breaks the flush messages. By moving all record time stamps 2 days into the past, the existing logic should work correctly with the new default broker configuration.

A similar thing happens in the relational smoke test, where data is emitted 4 days into the future. To avoid running into retention / compaction, the window retention time is increased for both tests.

Reviewers: Bruno Cadonna <bruno@confluent.io>, Bill Bejeck <bill@confluent.io>
2025-02-07 20:48:28 +01:00
Matthias J. Sax dcc27ec9c7
MINOR: update Kafka Streams `Topology` JavaDocs (#18814)
JavaDocs changes extracted from
https://github.com/apache/kafka/pull/18778 for 4.0 backport.

Reviewers: Bill Bejeck <bill@confluent.io>
2025-02-06 11:01:19 -08:00
Matthias J. Sax 53aa8dedea MINOR: fix NPE in KS `Topology` for new `AutoOffsetReset` (#18780)
Introduced via KIP-1106.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2025-02-03 17:30:49 -08:00
Joao Pedro Fonseca Dantas 37503cf42e MINOR: remove close from contextual processors javadoc
Reviewers: Matthias J. Sax <matthias@confluent.io>
2025-01-30 11:04:07 -08:00
Matthias J. Sax 7f9cf895d0 KAFKA-17402: DefaultStateUpdated should transite task atomically (#18607)
Reviewers: Bruno Cadonna <bruno@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>
2025-01-21 12:36:40 -08:00
Matthias J. Sax c6d452b635 KAFKA-17561: add processId tag to thread-state metric (#18581)
Part of KIP-1091.

Reviewers: Bill Bejeck <bill@confluent.io>
2025-01-21 08:10:46 -08:00
Jason Taylor caf0b67fbb KAFKA-16368: Update defaults for LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_DEFAULT and NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG (#18106)
Reviewers: Divij Vaidya <diviv@amazon.com>
2025-01-16 15:35:47 +01:00
Matthias J. Sax c438ee9166 MINOR: Kafka Streams doc updates for 4.0 release (#18488)
Reviewers: Bill Bejeck <bill@confluent.io>
2025-01-15 12:02:32 -08:00
Matthias J. Sax 8c3f0886a1 KAFKA-18479: RocksDBTimeOrderedKeyValueBuffer not initialized correctly (#18490)
RocksDBTimeOrderedKeyValueBuffer is not initialize with serdes provides
via Joined, but always uses serdes from StreamsConfig.

Reviewers: Bill Bejeck <bill@confluent.io>
2025-01-15 10:06:06 -08:00
Matthias J. Sax 6a9ba19a5f KAFKA-18476: KafkaStreams should swallow TransactionAbortedException (#18487)
TransactionAbortedException is a follow up error to a previous error,
and such a previous error would already be handled when
`producer.abortTransaction()` is called. Thus, a
TransactionAbortedException can just be silently swallowed.

Reviewers: Bill Bejeck <bill@confluent.io>
2025-01-14 18:15:10 -08:00
A. Sophie Blee-Goldman 0293cbdbfa KAFKA-18026: KIP-1112, clean up StatefulProcessorNode (#18195)
Final cleanup of StatefulProcessorNode after converting all stateful operators to adding state stores via implementing the #stores method.
2025-01-14 18:13:31 -08:00
Almog Gavra abf74de663 KAFKA-18026: KIP-1112 migrate KTableSuppressProcessorSupplier (#18150)
Migrates KTableSuppressProcessorSupplier to use the the ProcessorSupplier#stores() method

Reviewers: Guozhang Wang <guozhang.wang.us@gmail.com>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2025-01-14 18:13:10 -08:00
Matthias J. Sax 0186534a99 Revert "KAFKA-17411: Create local state Standbys on start (#16922)" and "KAFKA-17978: Fix invalid topology on Task assignment (#17778)"
This reverts commit 571f50817c.
This reverts commit a696b4d6f4.
2025-01-13 10:52:58 -08:00
Matthias J. Sax 6974007881 KAFKA-8116: Update Kafka Streams archetype for Java 11 (#18471)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Bill Bejeck <bill@confluent.io>
2025-01-10 15:47:48 -08:00
Almog Gavra 5393972ad7 MINOR: improve StreamThread periodic processing log (#18430)
The current log is really helpful, this PR adds a bit more information to that log to help debug some issues. In particular, it is interesting to be able to debug situations that have long intervals between polls. It also includes a reference to how long it has been since it last logged so you don't have to find the previous time it was logged to compute quick per-second ratios.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2025-01-09 11:03:03 -08:00
Almog Gavra 6d588f4207 KAFKA-18326: fix merge iterator with cache tombstones (#18287)
See https://issues.apache.org/jira/browse/KAFKA-18326 for more information. The main bug here is that in the old implementation, deleted cache entries would be skipped so long as they didn't equal the next store key, which resulted in potentially skipping tombstones for future keys in the store.

Reviewers: Guozhang Wang <guozhang.wang.us@gmail.com>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2025-01-07 23:05:08 -08:00
Matthias J. Sax eaab8b69b3 MINOR: cleanup JavaDocs for deprecation warnings (#18402)
Reviewers: Bill Bejeck <bbejeck@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-07 18:56:24 -08:00
A. Sophie Blee-Goldman f8a933a167 KAFKA-18026: KIP-1112, configure all StoreBuilder & StoreFactory layers (#18345)
Because of how we have to wrap StoreFactory and StoreBuilder layers on top of each other for various parts of the topology building process, we need to make sure both of these are capable of configuration and will check for & delegate to an underlying layer if it exists

Reviewers: Guozhang Wang <guozhang.wang.us@gmail.com>
2024-12-31 13:46:34 -06:00
Matthias J. Sax 1ab598b8ed KAFKA-13722: code cleanup after deprecated StateStore.init() was removed (#18249)
We remove the deprecated overload of StateStore.init() and thus do not
need to cast any longer. This PR removes all unnecessary casts, and
additionally cleans ups all related classed to reduce warnings.

Reviewers: Bill Bejeck <bill@confluent.io>
2024-12-20 15:05:23 -08:00
PoAn Yang e8060720c2 KAFKA-18262 Remove DefaultPartitioner and UniformStickyPartitioner (#18204)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-20 15:34:17 +08:00
Matthias J. Sax 8afe9b9176 KAFKA-12829: Remove old Processor and ProcessorSupplier interfaces (#18238)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Bill Bejeck <bill@confluent.io>
2024-12-19 22:27:37 -08:00
Matthias J. Sax be1fe2b7b6 KAFKA-12829: Remove deprecated Topology#addProcessor of old Processor API (#18154)
Reviewers: John Huang <pegasashjy@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Bill Bejeck <bill@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>
2024-12-19 16:07:21 -08:00
Matthias J. Sax 79de8ee1cb KAFKA-13722: Refactor Kafka Streams store interfaces (#18243)
Refactor Segments and TimestampedSegments to not use old
ProcessorContext any longer.

Reviewers: Bruno Cadonna <bruno@confluent.io>
2024-12-19 11:57:06 -08:00
Matthias J. Sax 56671891d6 KAFKA-17131: Refactor TimeDefinitions (#18241)
Refactor TimeDefintiions to not use old ProcessorContext any longer.

Reviewers: Bruno Cadonna <bruno@confluent.io>
2024-12-19 11:43:21 -08:00
Matthias J. Sax 1bdf98ec15 KAFKA-13722: Refactor SerdeGetter (#18242)
Reviewers: Bruno Cadonna <cadonna@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-18 16:32:31 -08:00
Rohan 3d86ded1ce KAFKA-18026: transition KTable#filter impl to use processor wrapper (#18205)
This patch transitions the KTable#filter implementation to provide the materialized store via the ProcessorSupplier so that it can be wrapped by the processor wrapper if the wrapper is configured

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2024-12-17 23:37:36 -08:00
Matthias J. Sax a1b3dfe07d MINOR: improve Kafka Streams config docs (#18087)
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2024-12-16 15:24:08 -08:00
Matthias J. Sax 218c510ffa MINOR: improving some internal comments (#18152)
Reviewers: Yaroslav Kutsela (@Serwios), Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2024-12-16 15:24:01 -08:00
A. Sophie Blee-Goldman bac8928521
KAFKA-18026: KIP-1112, migrate foreign-key joins to use ProcesserSupplier#stores (#18194)
Convert FKJ processors to implementing the #stores method

Reviewers: Guozhang Wang <guozhang.wang.us@gmail.com>
2024-12-15 20:00:42 -08:00
Rohan ed10fc63a9
KAFKA-18026: supply stores for KTable#mapValues using ProcessorSupplier#stores (#18155)
KAFKA-18026: supply stores for KTable#mapValues using ProcessorSupplier#stores

Reviewers: Guozhang Wang <guozhang.wang.us@gmail.com>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2024-12-14 20:18:49 -08:00
A. Sophie Blee-Goldman 91575892d2
HOTFIX: RocksDBMetricsRecorder#init should null check taskId (#18151)
Appears to be a typo in the code, since the error message indicates this check is for taskId being null, but instead we accidentally check the streams metrics twice

Reviewers: Matthias Sax <mjsax@apache.org>, runo Cadonna <cadonna@apache.org>, Lucas Brutschy <lbrutschy@confluent.io>, Bill Bejeck <bbejeck@gmail.com>
2024-12-13 20:36:08 -08:00
Lianet Magrans 84bc0c26ee
KAFKA-18224: Explicit group protocol setting in streams resetter (#18172)
Reviewers: Matthias J. Sax <matthias@confluent.io>
2024-12-13 14:31:50 -05:00