Skip to content

KAFKA-16260: Deprecate window.size.ms and window.inner.class.serde in StreamsConfig #18297

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 4 commits into from
Mar 31, 2025

Conversation

FrankYang0529
Copy link
Member

@FrankYang0529 FrankYang0529 commented Dec 22, 2024

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.

KIP-1020: https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=290982804
Reviewers: Lucas Brutschy [email protected]

@FrankYang0529 FrankYang0529 force-pushed the KAFKA-16260 branch 2 times, most recently from 2fb2f23 to 926b1cf Compare December 22, 2024 14:51
@FrankYang0529 FrankYang0529 requested a review from mjsax January 8, 2025 00:18
@FrankYang0529 FrankYang0529 force-pushed the KAFKA-16260 branch 3 times, most recently from 17f14fa to 3cef9bd Compare February 12, 2025 10:57
Copy link
Member

@lucasbru lucasbru left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the PR, @FrankYang0529 ! Overall this looks like a well-done PR to me. I left some comments.

@@ -430,6 +430,9 @@ <h5><a id="upgrade_servers_400_notable" href="#upgrade_servers_400_notable">Nota
<li> See <a href="https://cwiki.apache.org/confluence/x/B40ODg">KIP-890</a> and
<a href="https://cwiki.apache.org/confluence/x/8ItyEg">KIP-1050</a> for more details </li>
</ul>
<li>
The <code>window.size.ms</code> and <code>window.inner.serde.class</code> in stream config are deprecated.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This should not be in the 4.0 section.

/**
* {@code windowed.inner.class.serde}
*
* @deprecated since 4.0.0.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

4.1.0

/**
* {@code window.size.ms}
*
* @deprecated since 4.0.0.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

4.1.0

@@ -430,6 +430,9 @@ <h5><a id="upgrade_servers_400_notable" href="#upgrade_servers_400_notable">Nota
<li> See <a href="https://cwiki.apache.org/confluence/x/B40ODg">KIP-890</a> and
<a href="https://cwiki.apache.org/confluence/x/8ItyEg">KIP-1050</a> for more details </li>
</ul>
<li>
The <code>window.size.ms</code> and <code>window.inner.serde.class</code> in stream config are deprecated.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
The <code>window.size.ms</code> and <code>window.inner.serde.class</code> in stream config are deprecated.
The <code>window.size.ms</code> and <code>window.inner.serde.class</code> in `StreamsConfig` are deprecated. Use the corresponding string constants defined in `TimeWindowedSerializer`, `TimeWindowedDeserializer`, `SessionWindowedSerializer` and `SessionWindowedDeserializer` instead.

Something like that? Could also be the actual string constants. But it's good to make these notes actionable.

Serde<T> windowInnerClassSerde = null;
if (windowedInnerClassSerdeConfig != null) {
String serializerConfigFrom = WINDOWED_INNER_SERIALIZER_CLASS;
String windowedInnerSerializerClassConfig = (String) configs.get(WINDOWED_INNER_SERIALIZER_CLASS);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

variable name suggestion: serializerConfigValue

Also in other places. I think the key/value relationship between these two variables is helpful to understand this code.

final String windowedInnerClassSerdeConfig = (String) configs.get(StreamsConfig.WINDOWED_INNER_CLASS_SERDE);
Serde<T> windowInnerClassSerde = null;
if (windowedInnerClassSerdeConfig != null) {
String serializerConfigFrom = WINDOWED_INNER_SERIALIZER_CLASS;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

variable name suggestion: serializerConfigKey

Also in other places

import java.util.Map;

public class SessionWindowedDeserializer<T> implements Deserializer<Windowed<T>> {

/**
* Configuration key for the windowed inner deserializer class.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we put any more information into this (and the other) java doc comments about the constants? Since this is a public string constant now, the use of the constant should ideally be clear from the javadoc comments.

@FrankYang0529
Copy link
Member Author

@lucasbru Thanks for the review. I addressed all comments. The document update is like following:

Screenshot 2025-03-26 at 12 50 26 PM

@FrankYang0529 FrankYang0529 requested a review from lucasbru March 26, 2025 14:34
Copy link
Member

@lucasbru lucasbru left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM, thanks!

@lucasbru lucasbru merged commit c5f8ae0 into apache:trunk Mar 31, 2025
23 checks passed
@FrankYang0529 FrankYang0529 deleted the KAFKA-16260 branch March 31, 2025 13:08
janchilling pushed a commit to janchilling/kafka that referenced this pull request Apr 4, 2025
… StreamsConfig (apache#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 <[email protected]>
Signed-off-by: PoAn Yang <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants