mjsax commented on code in PR #20408:
URL: https://github.com/apache/kafka/pull/20408#discussion_r2302551322


##########
docs/streams/developer-guide/config-streams.html:
##########
@@ -543,7 +543,7 @@ <h4><a class="toc-backref" 
href="#id45">num.standby.replicas</a><a class="header
             <td colspan="2">Added to a windows maintainMs to ensure data is 
not deleted from the log prematurely. Allows for clock drift.</td>
             <td><code class="docutils literal"><span 
class="pre">86400000</span></code></td> (1 day)</td>
           </tr>
-          <tr class="row-odd"><td>window.size.ms</td>
+          <tr class="row-odd"><td>window.size.ms (Deprecated. See <a 
href="datatypes.html#window-serdes">Window Serdes</a> for alternatives.)</td>

Review Comment:
   It seems, `windowed.inner.class.serde` is missing in this table -- might be 
good to add for completeness, and as opportunity to link to the newly added 
section, similar to what you do here for `window.size.ms`.



##########
docs/streams/developer-guide/datatypes.html:
##########
@@ -163,6 +164,72 @@ <h3>JSON<a class="headerlink" href="#json" 
title="Permalink to this headline"></
         <p>As shown in the example, you can use JSONSerdes inner classes <code 
class="docutils literal"><span 
class="pre">Serdes.serdeFrom(&lt;serializerInstance&gt;, 
&lt;deserializerInstance&gt;)</span></code> to construct JSON compatible 
serializers and deserializers.
         </p>
       </div>
+      <div class="section" id="window-serdes">
+        <h3>Window Serdes<a class="headerlink" href="#window-serdes" 
title="Permalink to this headline"></a></h3>
+        <p>Apache Kafka Streams includes serde implementations for windowed 
keys in
+          its <code class="docutils literal"><span 
class="pre">kafka-streams</span></code> Maven artifact:</p>
+        <pre class="line-numbers"><code class="language-xml">&lt;dependency&gt;
+    &lt;groupId&gt;org.apache.kafka&lt;/groupId&gt;
+    &lt;artifactId&gt;kafka-streams&lt;/artifactId&gt;
+    &lt;version&gt;{{fullDotVersion}}&lt;/version&gt;
+&lt;/dependency&gt;</code></pre>
+        <p>This artifact provides the following windowed serde implementations 
under the package <a class="reference external" 
href="https://github.com/apache/kafka/blob/{{dotVersion}}/streams/src/main/java/org/apache/kafka/streams/kstream";>org.apache.kafka.streams.kstream</a>:</p>
+        <table border="1" class="docutils">
+          <colgroup>
+            <col width="17%" />
+            <col width="83%" />
+          </colgroup>
+          <thead valign="bottom">
+          <tr class="row-odd"><th class="head">Data type</th>
+            <th class="head">Serde</th>
+          </tr>
+          </thead>
+          <tbody valign="top">
+          <tr class="row-even"><td>Windowed&lt;T&gt; (Time Windows)</td>
+            <td><code class="docutils literal"><span class="pre">new 
WindowedSerdes.TimeWindowedSerde&lt;&gt;(innerSerde, windowSize)</span></code>, 
<code class="docutils literal"><span 
class="pre">WindowedSerdes.timeWindowedSerdeFrom(Class&lt;T&gt; type, long 
windowSize)</span></code></td>
+          </tr>
+          <tr class="row-odd"><td>Windowed&lt;T&gt; (Session Windows)</td>
+            <td><code class="docutils literal"><span class="pre">new 
WindowedSerdes.SessionWindowedSerde&lt;&gt;(innerSerde)</span></code>, <code 
class="docutils literal"><span 
class="pre">WindowedSerdes.sessionWindowedSerdeFrom(Class&lt;T&gt; 
type)</span></code></td>
+          </tr>
+          <tr class="row-even"><td>TimeWindowedSerializer&lt;T&gt;</td>
+            <td><code class="docutils literal"><span class="pre">new 
TimeWindowedSerializer&lt;&gt;(innerSerializer)</span></code></td>
+          </tr>
+          <tr class="row-odd"><td>TimeWindowedDeserializer&lt;T&gt;</td>
+            <td><code class="docutils literal"><span class="pre">new 
TimeWindowedDeserializer&lt;&gt;(innerDeserializer, 
windowSize)</span></code></td>
+          </tr>
+          <tr class="row-even"><td>SessionWindowedSerializer&lt;T&gt;</td>
+            <td><code class="docutils literal"><span class="pre">new 
SessionWindowedSerializer&lt;&gt;(innerSerializer)</span></code></td>
+          </tr>
+          <tr class="row-odd"><td>SessionWindowedDeserializer&lt;T&gt;</td>
+            <td><code class="docutils literal"><span class="pre">new 
SessionWindowedDeserializer&lt;&gt;(innerDeserializer)</span></code></td>
+          </tr>
+          </tbody>
+        </table>
+        <h4>Migration from Deprecated Configs</h4>

Review Comment:
   For regular code, which uses the constructors, and thus pass in `type` 
parameters (plus for time-windows the `size` parameter) nothing actually 
changes.
   
   And ever for command-line users, not much changes, because the die only 
"move" where the parameter name is defined, but we did not change the parameter 
name, and on command line, one type the parameter name directly: 
`windowed.inner.serializer.class`.
   
   Maybe we should have two sections: "Usage in Code" and do some code 
snippets, plus a "Usage in Command Line" showing some CLI commands passing in 
the inner class (and window size) via configs.
   
   Finally, we can have a small section `Deprecated Configs` just briefly 
mentioning, that `StreamsConfig#xxx` is deprecated in favor of 
`Yyy(De)Serializer#xxx`, without providing any further context. Nobody should 
need to rewrite code actually...



##########
docs/streams/developer-guide/datatypes.html:
##########
@@ -163,6 +164,72 @@ <h3>JSON<a class="headerlink" href="#json" 
title="Permalink to this headline"></
         <p>As shown in the example, you can use JSONSerdes inner classes <code 
class="docutils literal"><span 
class="pre">Serdes.serdeFrom(&lt;serializerInstance&gt;, 
&lt;deserializerInstance&gt;)</span></code> to construct JSON compatible 
serializers and deserializers.
         </p>
       </div>
+      <div class="section" id="window-serdes">
+        <h3>Window Serdes<a class="headerlink" href="#window-serdes" 
title="Permalink to this headline"></a></h3>
+        <p>Apache Kafka Streams includes serde implementations for windowed 
keys in
+          its <code class="docutils literal"><span 
class="pre">kafka-streams</span></code> Maven artifact:</p>
+        <pre class="line-numbers"><code class="language-xml">&lt;dependency&gt;
+    &lt;groupId&gt;org.apache.kafka&lt;/groupId&gt;
+    &lt;artifactId&gt;kafka-streams&lt;/artifactId&gt;
+    &lt;version&gt;{{fullDotVersion}}&lt;/version&gt;
+&lt;/dependency&gt;</code></pre>
+        <p>This artifact provides the following windowed serde implementations 
under the package <a class="reference external" 
href="https://github.com/apache/kafka/blob/{{dotVersion}}/streams/src/main/java/org/apache/kafka/streams/kstream";>org.apache.kafka.streams.kstream</a>:</p>
+        <table border="1" class="docutils">
+          <colgroup>
+            <col width="17%" />
+            <col width="83%" />
+          </colgroup>
+          <thead valign="bottom">
+          <tr class="row-odd"><th class="head">Data type</th>
+            <th class="head">Serde</th>
+          </tr>
+          </thead>
+          <tbody valign="top">
+          <tr class="row-even"><td>Windowed&lt;T&gt; (Time Windows)</td>

Review Comment:
   The data type is actually always `Windowed<T>` which is a wrapper around 
`Window` and `T`. I understand your intention but it seems this table is too 
redundant. Also the text says `provides the following windowed serde 
implementations`, above, but we also list `serializers`, and `deserializers`. I 
think we can leave the text, but maybe we should split it into two bullet point 
lists only?
   
   I would not list the constructors, but only name what it available. Also, 
all classed have generic type `T` for the inner class which we should add?
   ```
   This artifact provides the following windowed serde implementations under 
the package org.apache.kafka.streams.kstream:
   
   Serdes:
    - WindowedSerdes.TimeWindowedSerde&lt;T&gt;
     - WindowedSerdes.SessionWindowedSerde&lt;T&gt;
     
   Serializers:
    - TimeWindowedSerializer&lt;T&gt;
    - SessionWindowedSerializer&lt;T&gt;
    
   Deserializers:
    - TimeWindowedDeserializer&lt;T&gt;
    - SessionWindowedDeserializer&lt;T&gt;
   ```



##########
docs/streams/developer-guide/config-streams.html:
##########
@@ -1225,7 +1225,7 @@ <h4><a class="toc-backref" 
href="#id31">topology.optimization</a><a class="heade
         </div></blockquote>
     </div>
     <div class="section" id="windowed.inner.class.serde">

Review Comment:
   As we deprecated this config, might be better to remove this section 
entirely, and only link to the newly added section in datatypes.html instead?



##########
docs/streams/developer-guide/datatypes.html:
##########
@@ -163,6 +164,72 @@ <h3>JSON<a class="headerlink" href="#json" 
title="Permalink to this headline"></
         <p>As shown in the example, you can use JSONSerdes inner classes <code 
class="docutils literal"><span 
class="pre">Serdes.serdeFrom(&lt;serializerInstance&gt;, 
&lt;deserializerInstance&gt;)</span></code> to construct JSON compatible 
serializers and deserializers.
         </p>
       </div>
+      <div class="section" id="window-serdes">
+        <h3>Window Serdes<a class="headerlink" href="#window-serdes" 
title="Permalink to this headline"></a></h3>
+        <p>Apache Kafka Streams includes serde implementations for windowed 
keys in

Review Comment:
   ```suggestion
           <p>Apache Kafka Streams includes serde implementations for windowed 
types in
   ```
   While it's usually used for keys, there is not such entanglement, so using 
`types` is more generic.



-- 
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