Commit Graph

3100 Commits

Author SHA1 Message Date
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 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
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
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
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
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
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
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
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
Bill Bejeck 87a299a081
KAFKA-18826: Add global thread metrics (#18953)
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>
2025-04-02 09:17:49 -04:00
PoAn Yang c5f8ae0424
KAFKA-16260: Deprecate window.size.ms and window.inner.class.serde in StreamsConfig (#18297)
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>
2025-03-31 14:15:37 +02:00
Vikas Singh 56d1dc1b6e
MINOR: Use readable interface to parse requests (#19163)
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>
2025-03-26 10:13:13 -04:00
David Jacot 1ded681684
MINOR: Add 4.0.0 to streams system tests (#19239)
This patch adds 4.0.0 to streams system tests.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2025-03-21 01:41:09 -07:00
ClarkChen ec3c319c35
MINOR: Rewrite unchecked operations in Mock API (#19071)
We encountered unchecked or unsafe operations in
`GroupMetadataManagerTest.java`, `KTableImplTest.java`, and
`ConfigCommandIntegrationTest.java`.

* Rewrite getArgument of invocation in InvocationOnMock API because the
implementation of InvocationOnMock discards type anyway in in
`GroupMetadataManagerTest.java`.

* Remove unchecked annotations for using mock API without variable
assignment in `KTableImplTest.java`.

<img width="1422" alt="Screenshot 2025-03-02 at 8 50 55 AM"
src="https://github.com/user-attachments/assets/10ff1799-ebaa-499c-9acd-ca3b30484e6d"
/>
  
* Follow-up: https://github.com/mockito/mockito/issues/1609

Update on March 2.
* Fix unchecked cast for KTableImpl in `KTableImplTest.java`.
<img width="1259" alt="Screenshot 2025-03-02 at 5 17 47 PM"
src="https://github.com/user-attachments/assets/a5ffa3d7-4897-43ee-9b5f-26337e2560c5"
/>

Update on March 10.
* Use anyMap instead any for unchecked map type issues.
<img width="1691" alt="Screenshot 2025-03-10 at 9 36 38 AM"
src="https://github.com/user-attachments/assets/9aabc595-e7ba-4e04-81f6-f238d42af5a6"
/>

Pass all testing.
<img width="946" alt="Screenshot 2025-03-10 at 10 10 56 AM"
src="https://github.com/user-attachments/assets/793f67ea-09dc-44af-9d6c-de15531e9e72"
/>

Reviewers: TengYao Chi <kitingiao@gmail.com>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-18 00:47:34 +08:00
Matthias J. Sax 2181ddbb03
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 09:37:11 -07:00
Alieh Saeedi ca834d1174
MINOR: remove incorrect deprecated annotations from internal Streams classes (#19195)
Reviewers: Matthias J. Sax <matthias@confluent.io>
2025-03-12 17:49:00 -07:00
Bruno Cadonna 811761707c
MINOR: Downgrade log messages of swallowed exception to warnings (#19185)
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>
2025-03-11 16:43:04 +01:00
Lucas Brutschy fc2e3dfce9
MINOR: Disallow unused local variables (#18963)
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>
2025-03-10 09:37:35 +01:00
Lucas Brutschy 7fcee6f459
MINOR: transformValues test improvement (#19106)
Follow-up to 295760d3eb. Also check the
set of state stores as suggested in reviews.

Reviewers: Bruno Cadonna <cadonna@apache.org>
2025-03-06 17:54:30 +01:00
co63oc 3d7ac0c3d1
MINOR: Fix typos in multiple files (#19102)
Fix typos in multiple files

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-03-05 14:27:32 +00:00
Lucas Brutschy 295760d3eb
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:49:47 +01:00
Bruno Cadonna ff94c44e70
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 11:58:56 +01:00
Janindu Pathirana 2e6e5304c0
KAFKA-18168: Adding checkpointing for GlobalKTable during restoration and closing (#18752)
To address the issue of not creating a checkpoint file during the
restoring and closing process, called the
GlobalStateUpdateTask.flushState() method in
GlobalStateUpdateTask.initialize() and GlobalStateUpdateTask.close()
methods. This will flush the state and create a checkpoint file thereby,
avoiding the need to completely restore the entire state.

Reviewers: Alieh Saeedi <asaeedi@confluent.io>, Matthias J. Sax <matthias@confluent.io>
2025-03-02 12:16:48 -08:00
A. Sophie Blee-Goldman f20f299492
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:05:03 -08:00
Sebastien Viale 3ce5f23295
KAFKA-18023: Enforcing Explicit Naming for Kafka Streams Internal Topics (#18233)
Pull request to implement KIP-1111, aims to add a configuration that
prevents a Kafka Streams application from starting if any of its
internal topics have auto-generated names, thereby enforcing explicit
naming for all internal topics and enhancing the stability of the
application’s topology.

- Repartition Topics:

All repartition topics are created in the
KStreamImpl.createRepartitionedSource(...) static method. This method
either receives a name explicitly provided by the user or null and then
builds the final repartition topic name.

- Changelog Topics and State Store Names:

There are several scenarios where these are created:
  In the MaterializedInternal constructor.
  During KStream/KStream joins.
  During KStream/KTable joins with grace periods.
  With key-value buffers are used in suppressions.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Sophie Blee-Goldman <sophie@responsive.dev>
2025-02-24 11:41:42 +01:00
Matthias J. Sax acea35ddf3
MINOR: cleanup SinkNode generics (#18975)
Reviewers: Andrew Schofield <aschofield@confluent.io>, Bill Bejeck <bill@confluent.io>
2025-02-20 17:47:39 -08:00
Matthias J. Sax 9f23b25f6e
MINOR: fix Kafka Streams "smoke test" pass criteria (#18835)
Reviewers: Bill Bejeck <bill@confluent.io>, Bruno Cadonna <bruno@confluent.io>
2025-02-19 14:33:31 -08:00
Matthias J. Sax 538a60e1b3
MINOR: disallow rawtypes and fail build (#18877)
Cleanup code to avoid rawtype, and add suppressions where necessary.
Change the build to fail on rawtype warning.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2025-02-19 13:11:49 -08:00
Matthias J. Sax 900d81b345
MINOR: cleanup top level class JavaDocs for main interfaces of Kafka Streams DSL (4/N) (#18884)
Reviewers: Bill Bejeck <bill@confluent.io>
2025-02-18 16:22:18 -08:00
Matthias J. Sax 490ba8a8a3
MINOR: cleanup top level class JavaDocs for main interfaces of Kafka Streams DSL (3/N) (#18883)
Reviewers: Bill Bejeck <bill@confluent.io>
2025-02-18 16:21:52 -08:00