You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
KAFKA-16260: Deprecate window.size.ms and window.inner.class.serde in 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]>
Copy file name to clipboardExpand all lines: docs/upgrade.html
+8
Original file line number
Diff line number
Diff line change
@@ -36,6 +36,14 @@ <h6><a id="upgrade_410_notable" href="#upgrade_410_notable">Notable changes in 4
36
36
</li>
37
37
</ul>
38
38
</li>
39
+
<li><b>Stream</b>
40
+
<ul>
41
+
<li>
42
+
The <code>window.size.ms</code> and <code>window.inner.serde.class</code> in <code>StreamsConfig</code> are deprecated.
43
+
Use the corresponding string constants defined in <code>TimeWindowedSerializer</code>, <code>TimeWindowedDeserializer</code>, <code>SessionWindowedSerializer</code> and <code>SessionWindowedDeserializer</code> instead.
44
+
</li>
45
+
</ul>
46
+
</li>
39
47
</ul>
40
48
41
49
<h4><aid="upgrade_4_0_0"href="#upgrade_4_0_0">Upgrading to 4.0.0</a></h4>
Copy file name to clipboardExpand all lines: streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java
Copy file name to clipboardExpand all lines: streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/SelfJoinUpgradeIntegrationTest.java
Copy file name to clipboardExpand all lines: streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/SlidingWindowedKStreamIntegrationTest.java
Copy file name to clipboardExpand all lines: streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/TimeWindowedKStreamIntegrationTest.java
privatestaticfinalStringWINDOWED_INNER_CLASS_SERDE_DOC = " Default serializer / deserializer for the inner class of a windowed record. Must implement the " +
835
848
"<code>org.apache.kafka.common.serialization.Serde</code> interface. Note that setting this config in KafkaStreams application would result " +
836
849
"in an error as it is meant to be used only from Plain consumer client.";
837
850
838
-
/** {@code window.size.ms} */
851
+
/**
852
+
* {@code window.size.ms}
853
+
*
854
+
* @deprecated since 4.1.0.
855
+
* Use {@link TimeWindowedDeserializer#WINDOW_SIZE_MS_CONFIG} for {@link TimeWindowedDeserializer}.
0 commit comments