mjsax commented on a change in pull request #10573:
URL: https://github.com/apache/kafka/pull/10573#discussion_r619875764



##########
File path: docs/streams/upgrade-guide.html
##########
@@ -93,6 +95,12 @@ <h1>Upgrade Guide and API Changes</h1>
     </p>
 
     <h3><a id="streams_api_changes_300" 
href="#streams_api_changes_300">Streams API changes in 3.0.0</a></h3>
+    <p>
+      The <code>StreamsConfig.EXACTLY_ONCE</code> and 
<code>StreamsConfig.EXACTLY_ONCE_BETA</code> configs have been deprecated, and 
a new <code>StreamsConfig.EXACTLY_ONCE_V2</code> config has been
+      introduced. This is the same feature as eos-beta, but renamed to 
highlight its production-readiness. Users of exactly-once semantics should plan 
to migrate to the eos-v2 config and prepare for the removal of the deprecated 
configs in 4.0 or after at least a year

Review comment:
       I would be very bolt about it:
   ```
   We deprecated <code>processing.guarantee</code> configuration value 
<code>"exactly_once"</code>
   (for EOS version 1) in favor of the improved EOS version 2, formerly 
configured via
   <code>"exactly_once_beta</code>. To avoid the confusion about the term 
"beta" in the config value
   (it was never meant to imply it's not production ready), we furthermore 
renamed
   <code>"exactly_once_beta"</code> to <code>"exactly_once_v2"</code>. 
   ```
   
   Or something similar.

##########
File path: docs/streams/upgrade-guide.html
##########
@@ -93,6 +95,12 @@ <h1>Upgrade Guide and API Changes</h1>
     </p>
 
     <h3><a id="streams_api_changes_300" 
href="#streams_api_changes_300">Streams API changes in 3.0.0</a></h3>
+    <p>
+      The <code>StreamsConfig.EXACTLY_ONCE</code> and 
<code>StreamsConfig.EXACTLY_ONCE_BETA</code> configs have been deprecated, and 
a new <code>StreamsConfig.EXACTLY_ONCE_V2</code> config has been

Review comment:
       Well, we do deprecate `StreamsConfig.EXACTLY_ONCE`, too, but user might 
just do `config.put("processing.guarantee", "exactly_once");` (or have a config 
file with `"exactly_once"`) in it. To me, the main change is that the config 
itself is deprecated and the deprecation of variable is just a "side effect".

##########
File path: docs/streams/upgrade-guide.html
##########
@@ -93,6 +95,12 @@ <h1>Upgrade Guide and API Changes</h1>
     </p>
 
     <h3><a id="streams_api_changes_300" 
href="#streams_api_changes_300">Streams API changes in 3.0.0</a></h3>
+    <p>
+      The <code>StreamsConfig.EXACTLY_ONCE</code> and 
<code>StreamsConfig.EXACTLY_ONCE_BETA</code> configs have been deprecated, and 
a new <code>StreamsConfig.EXACTLY_ONCE_V2</code> config has been
+      introduced. This is the same feature as eos-beta, but renamed to 
highlight its production-readiness. Users of exactly-once semantics should plan 
to migrate to the eos-v2 config and prepare for the removal of the deprecated 
configs in 4.0 or after at least a year
+      from the release of 3.0, whichever comes last. Note that eos-v2 requires 
broker version 2.5 or higher, like eos-beta, so users should begin to upgrade 
their kafka cluster if necessary. See

Review comment:
       Fair enough.

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
##########
@@ -603,7 +606,7 @@ boolean runLoop() {
                     log.error("Shutting down because the Kafka cluster seems 
to be on a too old version. " +
                               "Setting {}=\"{}\" requires broker version 2.5 
or higher.",
                           StreamsConfig.PROCESSING_GUARANTEE_CONFIG,
-                          EXACTLY_ONCE_BETA);
+                          StreamsConfig.EXACTLY_ONCE_V2);

Review comment:
       SGTM -- if the required code changes to get the actual value are to 
much, I am fine with hard-coding the value, too.

##########
File path: 
streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java
##########
@@ -111,6 +113,7 @@
 
     private final StringSerializer stringSerializer = new StringSerializer();
     private final ByteArraySerializer byteArraySerializer = new 
ByteArraySerializer();
+    private final UUID processId = UUID.randomUUID();

Review comment:
       > only for eos-v2 for some reason
   
   That is weird -- if `StreamsProducer` requires the `processID` it should 
have required it for `_beta` already? Would be good to understand -- maybe we 
unmasked a bug?

##########
File path: 
streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java
##########
@@ -126,10 +129,10 @@ public void setup() {
         clientSupplier.setCluster(cluster);
         streamsProducer = new StreamsProducer(
             config,
-            "threadId",
+            processId + "-StreamThread-1",

Review comment:
       This PR does not change `StreamsProducer` so this parsing should have 
happened for `_beta` already -- what do I miss?




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

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


Reply via email to