lucasbru commented on code in PR #18297:
URL: https://github.com/apache/kafka/pull/18297#discussion_r2010879588


##########
streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindowedDeserializer.java:
##########
@@ -23,10 +23,20 @@
 import org.apache.kafka.streams.StreamsConfig;
 import org.apache.kafka.streams.state.internals.SessionKeySchema;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import java.util.Map;
 
 public class SessionWindowedDeserializer<T> implements 
Deserializer<Windowed<T>> {
 
+    /**
+     * Configuration key for the windowed inner deserializer class.

Review Comment:
   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.



##########
docs/upgrade.html:
##########
@@ -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.

Review Comment:
   This should not be in the 4.0 section.
   



##########
docs/upgrade.html:
##########
@@ -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.

Review Comment:
   ```suggestion
                       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.



##########
streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindowedSerializer.java:
##########
@@ -38,32 +48,42 @@ public TimeWindowedSerializer(final Serializer<T> inner) {
         this.inner = inner;
     }
 
-    @SuppressWarnings("unchecked")
+    @SuppressWarnings({"deprecation", "unchecked"})
     @Override
     public void configure(final Map<String, ?> configs, final boolean isKey) {
-        final String windowedInnerClassSerdeConfig = (String) 
configs.get(StreamsConfig.WINDOWED_INNER_CLASS_SERDE);
-        Serde<T> windowInnerClassSerde = null;
-        if (windowedInnerClassSerdeConfig != null) {
+        String serializerConfigFrom = WINDOWED_INNER_SERIALIZER_CLASS;

Review Comment:
   variable name suggestion: serializerConfigKey
   
   Also in other places



##########
streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java:
##########
@@ -823,13 +827,28 @@ public class StreamsConfig extends AbstractConfig {
         + CONFIG_ERROR_MSG
         + "\"NO_OPTIMIZATION\" by default.";
 
-    /** {@code windowed.inner.class.serde} */
+    /**
+     * {@code windowed.inner.class.serde}
+     *
+     * @deprecated since 4.0.0.

Review Comment:
   4.1.0



##########
streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java:
##########
@@ -823,13 +827,28 @@ public class StreamsConfig extends AbstractConfig {
         + CONFIG_ERROR_MSG
         + "\"NO_OPTIMIZATION\" by default.";
 
-    /** {@code windowed.inner.class.serde} */
+    /**
+     * {@code windowed.inner.class.serde}
+     *
+     * @deprecated since 4.0.0.
+     * Use {@link TimeWindowedSerializer#WINDOWED_INNER_SERIALIZER_CLASS} for 
{@link TimeWindowedSerializer}.
+     * Use {@link TimeWindowedDeserializer#WINDOWED_INNER_DESERIALIZER_CLASS} 
for {@link TimeWindowedDeserializer}.
+     * Use {@link SessionWindowedSerializer#WINDOWED_INNER_SERIALIZER_CLASS} 
for {@link SessionWindowedSerializer}.
+     * Use {@link 
SessionWindowedDeserializer#WINDOWED_INNER_DESERIALIZER_CLASS} for {@link 
SessionWindowedDeserializer}.
+     */
+    @Deprecated
     public static final String WINDOWED_INNER_CLASS_SERDE = 
"windowed.inner.class.serde";
     private static final String WINDOWED_INNER_CLASS_SERDE_DOC = " Default 
serializer / deserializer for the inner class of a windowed record. Must 
implement the " +
         "<code>org.apache.kafka.common.serialization.Serde</code> interface. 
Note that setting this config in KafkaStreams application would result " +
         "in an error as it is meant to be used only from Plain consumer 
client.";
 
-    /** {@code window.size.ms} */
+    /**
+     * {@code window.size.ms}
+     *
+     * @deprecated since 4.0.0.

Review Comment:
   4.1.0



##########
streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindowedSerializer.java:
##########
@@ -38,32 +48,42 @@ public TimeWindowedSerializer(final Serializer<T> inner) {
         this.inner = inner;
     }
 
-    @SuppressWarnings("unchecked")
+    @SuppressWarnings({"deprecation", "unchecked"})
     @Override
     public void configure(final Map<String, ?> configs, final boolean isKey) {
-        final String windowedInnerClassSerdeConfig = (String) 
configs.get(StreamsConfig.WINDOWED_INNER_CLASS_SERDE);
-        Serde<T> windowInnerClassSerde = null;
-        if (windowedInnerClassSerdeConfig != null) {
+        String serializerConfigFrom = WINDOWED_INNER_SERIALIZER_CLASS;
+        String windowedInnerSerializerClassConfig = (String) 
configs.get(WINDOWED_INNER_SERIALIZER_CLASS);

Review Comment:
   variable name suggestion: serializerConfigValue
   
   Also in other places. I think the key/value relationship between these two 
variables is helpful to understand this code. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to