Commit Graph

3159 Commits

Author SHA1 Message Date
Matthias J. Sax 738bd928f1
MINOR: cleanup JavaDocs for deprecation warnings (#18402)
Reviewers: Bill Bejeck <bbejeck@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-07 09:18:33 +00:00
Ismael Juma 409a43eff7
MINOR: Collection/Option usage simplification via methods introduced in Java 9 & 11 (#18305)
Relevant methods:
1. `List.of`, `Set.of`, `Map.of` and similar (introduced in Java 9)
2. Optional: `isEmpty` (introduced in Java 11), `stream` (introduced in Java 9).

Reviewers: Mickael Maison <mimaison@users.noreply.github.com>
2025-01-03 16:13:39 -08:00
Ismael Juma 73ab7ee4ea
MINOR: Use `Files.readString/writeString` and `String.repeat` to simplify code (#18372)
The 3 methods were introduced in Java 11.

Reviewers: Divij Vaidya <diviv@amazon.com>
2025-01-02 17:50:27 -08:00
A. Sophie Blee-Goldman 86b552835c
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 11:45:29 -08:00
Matthias J. Sax 8c21857877
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 14:31:44 -08:00
PoAn Yang 753a003480
KAFKA-18262 Remove DefaultPartitioner and UniformStickyPartitioner (#18204)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-20 15:33:51 +08:00
Matthias J. Sax f7ae4e8b71
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:26:17 -08:00
Matthias J. Sax 9aa900f076
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:05:16 -08:00
Matthias J. Sax b3b40bb77b
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:55:57 -08:00
Matthias J. Sax fe9847dd4a
KAFKA-17131: Refactor TimeDefinitions (#18241)
Refactor TimeDefintiions to not use old ProcessorContext any longer.

Reviewers: Bruno Cadonna <bruno@confluent.io>
2024-12-19 11:40:43 -08:00
Bruno Cadonna a0291a8d50
MINOR: Fix flaky state updater test (#18253)
The tests are flaky because the tests end before the verified calls
are executed. This happens because the state updater thread executes
the verified calls, but the thread that executes the
tests with the verifications is a different thread.

This commit fixes the flaky tests by enusring that the calls were
performed by the state updater by either shutting down the state
updater or waiting for the condition.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Matthias J. Sax <matthias@confluent.io>
2024-12-19 09:02:38 +01:00
Matthias J. Sax e551cb7bb3
KAFKA-13722: Refactor SerdeGetter (#18242)
Reviewers: Bruno Cadonna <cadonna@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-18 19:01:46 +08:00
Almog Gavra 4bcbf9fae7
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>
2024-12-17 23:45:49 -08:00
Rohan 501da383fa
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:32:24 -08:00
Matthias J. Sax 16f1aa84df
MINOR: improve Kafka Streams config docs (#18087)
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2024-12-16 15:21:52 -08:00
Matthias J. Sax 85c43e49e3
MINOR: improving some internal comments (#18152)
Reviewers: Yaroslav Kutsela (@Serwios), Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2024-12-16 15:21:03 -08:00
David Jacot 84049369c1
MINOR: Bump trunk to 4.1.0-SNAPSHOT (#18213)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-12-16 16:47:13 +01: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
TengYao Chi b37b89c668
KAFKA-9366 Upgrade log4j to log4j2 (#17373)
This pull request replaces Log4j with Log4j2 across the entire project, including dependencies, configurations, and code. The notable changes are listed below:

1. Introduce Log4j2 Instead of Log4j
2. Change Configuration File Format from Properties to YAML
3. Adds warnings to notify users if they are still using Log4j properties, encouraging them to transition to Log4j2 configurations

Co-authored-by: Lee Dongjin <dongjin@apache.org>

Reviewers: Luke Chen <showuon@gmail.com>, Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-14 01:14:31 +08:00
A. Sophie Blee-Goldman ef2e4600f3
KAFKA-18026: KIP-1112, migrate stream-stream joins to use ProcesserSupplier#stores (#18111)
Covers wrapping of processors and state stores for KStream-KStream joins.

Includes self-joins and the spurious results fix optimization

Reviewers: Guozhang Wang <guozhang.wang.us@gmail.com>
2024-12-12 14:54:58 -08:00
Almog Gavra 9b776ffc50
KAFKA-18026: KIP-1112 convert StreamToTableNode (#18149)
Covers wrapping of processors and state stores for StreamToTableSource

Reviewers: Guozhang Wang <guozhang.wang.us@gmail.com>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2024-12-12 14:52:21 -08:00
Lianet Magrans 7a64623e40
Set protocol for streams tests (#18160)
Reviewers: Bill Bejeck <bill@confluent.io>
2024-12-12 13:33:43 -05:00
Matthias J. Sax a0a501952b
MINOR: improve Kafka Streams metrics documentation (#17900)
Reviewers: Bruno Cadonna <bruno@confluent.io>, Guozhang Wang <guozhang.wang.us@gmail.com>
2024-12-11 18:34:43 -08:00
Almog Gavra 21563380f3
KAFKA-18026: KIP-1112, migrate table-table joins to use ProcesserSuppliers#stores (#18048)
Covers wrapping of processors and state stores for KTable-KTable joins

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Guozhang Wang <guozhang.wang.us@gmail.com>
2024-12-11 17:37:34 -08:00
Matthias J. Sax 6cdb8c352a
KAFKA-18015: add byDuration auto.offset.reset to Kafka Streams (#18115)
Part of KIP-1106.

Adds support for "by_duration" and "none" reset strategy
to the Kafka Streams runtime.

Reviewers: Bill Bejeck <bill@confluent.io>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2024-12-11 15:12:16 -08:00
Matthias J. Sax 990c8c750c
MINOR: remove old procesor API MockInternalProcessorContext (#18103)
Reviewers: Bill Bejeck <bill@confluent.io>
2024-12-11 15:09:13 -08:00
Matthias J. Sax ab2facca58
KAFKA-12829: Remove deprecated KStream.process() for old Processor API (#18088)
Reviewers: Bill Bejeck <bill@confluent.io>
2024-12-11 14:28:47 -08:00
KApolinario1120 d83f09d014
KAFKA-18015: Add support for duration based offset reset strategy to Kafka Streams (#17973)
Part of KIP-1106.

Adds the public APIs to Kafka Streams, to support the the newly added "by_duration" reset policy,
plus adds the missing "none" reset policy. Deprecates the enum `Topology.AutoOffsetReset` and
all related methods, and replaced them with new overload using the new `AutoOffsetReset` class.

Co-authored-by: Matthias J. Sax <matthias@confluent.io>

Reviewers: Matthias J. Sax <matthias@confluent.io>, Bill Bejeck <bill@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>
2024-12-11 10:47:25 -08:00
mingdaoy 4603f7495e
KAFKA-18030 Remove old upgrade-system-tests modules (#17843)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-10 11:19:14 +08:00
TengYao Chi e8837465a5
KAFKA-18067: Kafka Streams can leak Producer client under EOS (#17931)
To avoid leaking producers, we should add a 'closedflag toStreamProducer` indicating whether we should reset prouder.

Reviewers: Guozhang Wang <guozhang.wang.us@gmail.com>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2024-12-09 16:12:05 -08:00
Joao Pedro Fonseca Dantas d5c2029434
KAFKA-16339: [4/4 KStream#flatTransformValues] Remove Deprecated "transformer" methods and classes (#17882)
Reviewer: Matthias J. Sax <matthias@confluent.io>
2024-12-08 20:10:11 -08:00
yx9o 38e727fe4d
KAFKA-17864: add descriptions to fields in the agreement (#17681)
Improve descriptive information in Kafka protocol documentation.

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>
2024-12-07 18:47:11 +00:00
Andrew Schofield e7d986e48c
KAFKA-17550: DescribeGroups v6 exploitation (#17706)
This PR introduces the DescribeGroups v6 API as part of KIP-1043. This adds an error message for the described groups so that it is possible to get some context on the error. It also changes the behaviour for when the group ID cannot be found but returning error code GROUP_ID_NOT_FOUND rather than NONE.

Reviewers: David Jacot <djacot@confluent.io>
2024-12-05 23:12:24 -08:00
A. Sophie Blee-Goldman 09e8fa2dbe
KAFKA-18026: KIP-1112, migrate stream-table joins to use ProcesserSupplier#stores (#18047)
Covers wrapping of processors and state stores for KStream-KTable joins

Reviewers: Almog Gavra <almog@responsive.dev>, Guozhang Wang <guozhang.wang.us@gmail.com>
2024-12-05 10:06:11 -08:00
Ken Huang 2b43c49f51
KAFKA-18050 Upgrade the checkstyle version to 10.20.2 (#17999)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-05 10:59:18 +08:00
A. Sophie Blee-Goldman 31d97bc3c9
KAFKA-18026: KIP-1112, skip re-registering aggregate stores in StatefulProcessorNode (#18015)
Minor followup to #17929 based on this discussion

Also includes some very minor refactoring/renaming on the side. The only real change is in the KGroupedStreamImpl class

Reviewers: Guozhang Wang <guozhang.wang.us@gmail.com>
2024-12-03 22:18:55 -08:00
Peter Lee c76fb5cb9b
KAFKA-17893: Support record keys in the foreignKeyExtractor argument of KTable foreign join (#17756)
Currently, KTable foreign key joins only allow extracting the foreign key from the value of the source record. This forces users to duplicate data that might already exist in the key into the value when the foreign key needs to be derived from both the key and value. This leads to:

- Data duplication
- Additional storage overhead
- Potential data inconsistency if the duplicated data gets out of sync
- Less intuitive API when the foreign key is naturally derived from both key and value

This change allows user to extract the foreign key from the key and value of the source record.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2024-12-03 17:34:13 +01:00
A. Sophie Blee-Goldman 184b64fb41
KAFKA-18026: migrate KStream and KTable aggregates to use ProcesserSupplier#stores (#17929)
As part of KIP-1112, to maximize the utility of the new ProcessorWrapper, we need to migrate the DSL operators to the new method of attaching state stores by implementing ProcessorSupplier#stores, which makes these stores available for inspection by the user's wrapper.

This PR covers the aggregate operator for both KStream and KTable.


Reviewers: Guozhang Wang <guozhang.wang.us@gmail.com>, Rohan Desai <rohan@responsive.dev>
2024-12-03 02:09:43 -08:00
TengYao Chi 6fd951a9c0
KAFKA-17610 Drop alterConfigs (#18002)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-02 23:26:06 +08:00
Almog Gavra 5243fb9a7d
KAFKA-18026: migrate KTableSource to use ProcesserSupplier#stores (#17903)
This PR is part of the implementation for KIP-1112 (KAFKA-18026). In order to have DSL operators be properly wrapped by the interface suggestion in 1112, we need to make sure they all use the ConnectedStoreProvider#stores method to connect stores instead of manually calling addStateStore.

This is a refactor only, there is no new behaviors.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2024-11-27 14:04:27 -08:00
Bill Bejeck d334f60944
MINOR: KStreamRepartitionIntegrationTest bug (#17963)
The KStreamRepartitionIntegrationTest.shouldThrowAnExceptionWhenNumberOfPartitionsOfRepartitionOperationDoNotMatchSourceTopicWhenJoining test was taking two minutes due not reaching an expected condition. By updating to have the StreamsUncaughtExceptionHandler return a response of SHUTDOWN_CLIENT the expected ERROR state is now reached. The root cause was using the Thread.UncaughtExceptionHandler to handle the exception.

Without this fix, the test takes 2 minutes to run, and now it's 1 second.

Reviewers: Matthias Sax <mjsax@apache.org>
2024-11-27 16:08:05 -05:00
Joao Pedro Fonseca Dantas 3f834781a4
KAFKA-12844: clean up TaskId (#17904)
Rename topicGroupId as subtopology.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2024-11-26 17:06:36 -08:00
Matthias J. Sax f5d712396b
MINOR: fix warnings in Kafka Streams state store tests (#17855)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-27 01:54:21 +08:00
Matthias J. Sax 95947d2f58
KAFKA-17299: add unit tests for previous fix (#17919)
https://github.com/apache/kafka/pull/17899 fixed the issue, but did not
add any unit tests.

Reviewers: Bill Bejeck <bill@confluent.io>
2024-11-25 12:03:57 -08:00
Kaushik Raina 7908a4838b
Fix long running RangeQueryIntegrationTest. (#17933)
Noticed that RangeQueryIntegrationTest is taking ~approx 20 - 30min to run
Upon deep dive in logs, noticed that there were error for consumer rebalancing and test was stuck in loop
Seems like due to same application.id across tests, Kafka Streams application is failing to track its state correctly across rebalances.

Reviewers: Bill Bejeck <bbejeck@apache.org>
2024-11-25 11:42:02 -05:00
Andrew Schofield d17a149205
KAFKA-17956 Remove Admin.listShareGroups (#17912)
KIP-1043 introduced Admin.listGroups as the way to list all types of groups. As a result, Admin.listShareGroups has been removed. This PR is the final step of the removal.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-25 22:05:35 +08:00
Manikumar Reddy 3268435fd6
KAFKA-18013: Add AutoOffsetResetStrategy internal class (#17858)
- Deprecates OffsetResetStrategy enum
- Adds new internal class AutoOffsetResetStrategy
- Replaces all OffsetResetStrategy enum usages with AutoOffsetResetStrategy
- Deprecate old/Add new constructors to MockConsumer

 Reviewers: Andrew Schofield <aschofield@confluent.io>, Matthias J. Sax <matthias@confluent.io>
2024-11-25 19:11:12 +05:30