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



##########
File path: 
clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
##########
@@ -642,9 +647,13 @@ public void beginTransaction() throws 
ProducerFencedException {
      *         to the partition leader. See the exception for more details
      * @throws KafkaException if the producer has encountered a previous fatal 
or abortable error, or for any
      *         other unexpected error
+     *
+     * @deprecated Since 3.0.0, will be removed in 4.0. Use {@link 
#sendOffsetsToTransaction(Map, ConsumerGroupMetadata)} instead.
      */
+    @Deprecated
     public void sendOffsetsToTransaction(Map<TopicPartition, 
OffsetAndMetadata> offsets,
                                          String consumerGroupId) throws 
ProducerFencedException {
+        log.warn("This method has been deprecated and will be removed in 4.0, 
please use #sendOffsetsToTransaction(Map, ConsumerGroupMetadata) instead");

Review comment:
       Do we really need to log a WAR? We never did anything link this in KS 
code base in the past. Marking the method as `@Deprecated` should be sufficient 
IMHO? Or it such a WARN log custom in client code base?

##########
File path: clients/src/main/java/org/apache/kafka/clients/producer/Producer.java
##########
@@ -49,7 +49,10 @@
 
     /**
      * See {@link KafkaProducer#sendOffsetsToTransaction(Map, String)}
+     *
+

Review comment:
       nit: needs cleanup

##########
File path: docs/streams/core-concepts.html
##########
@@ -291,16 +291,18 @@ <h2 class="anchor-heading"><a 
id="streams_processing_guarantee" class="anchor-li
         commits on the input topic offsets, updates on the state stores, and 
writes to the output topics will be completed atomically instead of treating 
Kafka as an external system that may have side-effects.
         For more information on how this is done inside Kafka Streams, see <a 
href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-129%3A+Streams+Exactly-Once+Semantics";>KIP-129</a>.<br
 />
 
-        As of the 2.6.0 release, Kafka Streams supports an improved 
implementation of exactly-once processing, named "exactly-once beta", 
+        As of the 2.6.0 release, Kafka Streams supports an improved 
implementation of exactly-once processing, named "exactly-once v2",
         which requires broker version 2.5.0 or newer.
         This implementation is more efficient, because it reduces client and 
broker resource utilization, like client threads and used network connections, 
         and it enables higher throughput and improved scalability.
+        As of the 3.0.0 release, the old "alpha" version of exactly-once has 
been deprecated. Users are encouraged to use exactly-once v2 for

Review comment:
       We never used "alpha" anywhere in public, so might be better to avoid 
this term at all?
   
   `old "alpha" -> "first"

##########
File path: docs/streams/developer-guide/config-streams.html
##########
@@ -667,12 +668,14 @@ <h4><a class="toc-backref" 
href="#id30">probing.rebalance.interval.ms</a><a clas
           <span 
id="streams-developer-guide-processing-guarantee"></span><h4><a 
class="toc-backref" href="#id25">processing.guarantee</a><a class="headerlink" 
href="#processing-guarantee" title="Permalink to this headline"></a></h4>
           <blockquote>
             <div>The processing guarantee that should be used.
-              Possible values are <code class="docutils literal"><span 
class="pre">"at_least_once"</span></code> (default),
-              <code class="docutils literal"><span 
class="pre">"exactly_once"</span></code> (for EOS version 1),
-              and <code class="docutils literal"><span 
class="pre">"exactly_once_beta"</span></code> (for EOS version 2).
-              Using <code class="docutils literal"><span 
class="pre">"exactly_once"</span></code> requires broker
-              version 0.11.0 or newer, while using <code class="docutils 
literal"><span class="pre">"exactly_once_beta"</span></code>
-              requires broker version 2.5 or newer.
+              Possible values are <code class="docutils literal"><span 
class="pre">"at_least_once"</span></code> (default)
+              and <code class="docutils literal"><span 
class="pre">"exactly_once_v2"</span></code> (for EOS version 2).
+              Deprecated config options are <code class="docutils 
literal"><span class="pre">"exactly_once"</span></code> (for EOS alpha),

Review comment:
       Do we need to elaborate on all deprecated configs?

##########
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:
       Might be good to highlight at `beta -> v2` is just a renaming ?

##########
File path: docs/streams/upgrade-guide.html
##########
@@ -53,17 +53,19 @@ <h1>Upgrade Guide and API Changes</h1>
     </ul>
 
     <p>
-        Starting in Kafka Streams 2.6.x, a new processing mode is available, 
named EOS version 2, which is configurable by setting 
-        <code>processing.guarantee</code> to <code>"exactly_once_beta"</code>.
-        <b>NOTE:</b> The <code>"exactly_once_beta"</code> processing mode is 
ready for production (<i>i.e.</i>, it's not "beta" software). 
+        Starting in Kafka Streams 2.6.x, a new processing mode is available, 
named EOS version 2. This can be configured
+        by setting <code>StreamsConfig.PROCESSING_GUARANTEE</code> to 
<code>StreamsConfig.EXACTLY_ONCE_V2</code> for
+        application versions 3.0+, or setting it to 
<code>StreamsConfig.EXACTLY_ONCE_BETA</code> for versions between 2.6 and 3.0.
         To use this new feature, your brokers must be on version 2.5.x or 
newer.
-        A switch from <code>"exactly_once"</code> to 
<code>"exactly_once_beta"</code> (or the other way around) is
-        only possible if the application is on version 2.6.x.
-        If you want to upgrade your application from an older version and 
enable this feature,
-        you first need to upgrade your application to version 2.6.x, staying 
on <code>"exactly_once"</code>,
-        and then do second round of rolling bounces to switch to 
<code>"exactly_once_beta"</code>.
-        For a downgrade, do the reverse: first switch the config from 
<code>"exactly_once_beta"</code> to
-        <code>"exactly_once"</code> to disable the feature in your 2.6.x 
application.
+        If you want to upgrade your EOS application from an older version and 
enable this feature,
+        you first need to upgrade your application to version 2.6.x, staying 
on <code>StreamsConfig.EXACTLY_ONCE</code>,

Review comment:
       `you first need to upgrade your application to version 2.6.x`
   
   Why would that be required? On can go to `3.0` directly from my 
understanding?

##########
File path: docs/streams/developer-guide/config-streams.html
##########
@@ -293,8 +293,9 @@ <h4><a class="toc-backref" 
href="#id5">bootstrap.servers</a><a class="headerlink
           </tr>
           <tr class="row-even"><td>processing.guarantee</td>
             <td>Medium</td>
-            <td colspan="2">The processing mode. Can be either <code 
class="docutils literal"><span class="pre">"at_least_once"</span></code> 
(default),
-              <code class="docutils literal"><span 
class="pre">"exactly_once"</span></code> (for EOS version 1), or <code 
class="docutils literal"><span class="pre">"exactly_once_beta"</span></code> 
(for EOS version 2)</td>.
+            <td colspan="2">The processing mode. Can be either <code 
class="docutils literal"><span class="pre">"at_least_once"</span></code> 
(default)
+              or <code class="docutils literal"><span 
class="pre">"exactly_once_v2"</span></code> (for EOS version 2). Deprecated 
config options are

Review comment:
       We should add `requires broker version 2.5 or newer`

##########
File path: docs/streams/developer-guide/config-streams.html
##########
@@ -293,8 +293,9 @@ <h4><a class="toc-backref" 
href="#id5">bootstrap.servers</a><a class="headerlink
           </tr>
           <tr class="row-even"><td>processing.guarantee</td>
             <td>Medium</td>
-            <td colspan="2">The processing mode. Can be either <code 
class="docutils literal"><span class="pre">"at_least_once"</span></code> 
(default),
-              <code class="docutils literal"><span 
class="pre">"exactly_once"</span></code> (for EOS version 1), or <code 
class="docutils literal"><span class="pre">"exactly_once_beta"</span></code> 
(for EOS version 2)</td>.
+            <td colspan="2">The processing mode. Can be either <code 
class="docutils literal"><span class="pre">"at_least_once"</span></code> 
(default)
+              or <code class="docutils literal"><span 
class="pre">"exactly_once_v2"</span></code> (for EOS version 2). Deprecated 
config options are
+              <code class="docutils literal"><span 
class="pre">"exactly_once"</span></code> (for EOS alpha) and <code 
class="docutils literal"><span class="pre">"exactly_once_beta"</span></code> 
(for EOS beta)</td>.

Review comment:
       don't use `alpha`; maybe just say `(requires broker version 0.11 or 
newer)

##########
File path: streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
##########
@@ -1010,18 +1026,51 @@ public StreamsConfig(final Map<?, ?> props) {
     protected StreamsConfig(final Map<?, ?> props,
                             final boolean doLog) {
         super(CONFIG, props, doLog);
-        eosEnabled = StreamThread.eosEnabled(this);
+        eosEnabled = eosEnabled();
+
+        final String processingModeConfig = 
getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG);
+        if (processingModeConfig.equals(EXACTLY_ONCE)) {
+            log.warn("Configuration parameter `{}` is deprecated and will be 
removed in the 4.0.0 release. " +
+                         "Please use `{}` instead. Note that this requires 
broker version 2.5+ so you should prepare "
+                         + "to upgrade your brokers if necessary.", 
EXACTLY_ONCE, EXACTLY_ONCE_V2);
+        }
+        if (processingModeConfig.equals(EXACTLY_ONCE_BETA)) {
+            log.warn("Configuration parameter `{}` is deprecated and will be 
removed in the 4.0.0 release. " +
+                         "Please use `{}` instead.", EXACTLY_ONCE_BETA, 
EXACTLY_ONCE_V2);
+        }
+
         if (props.containsKey(RETRIES_CONFIG)) {
-            log.warn("Configuration parameter `{}` is deprecated and will be 
removed in 3.0.0 release.", RETRIES_CONFIG);
+            log.warn("Configuration parameter `{}` is deprecated and will be 
removed in the 4.0.0 release.", RETRIES_CONFIG);
+        }
+    }
+
+    public ProcessingMode processingMode() {
+        if 
(EXACTLY_ONCE.equals(getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG))) {
+            return StreamThread.ProcessingMode.EXACTLY_ONCE_ALPHA;
+        } else if 
(EXACTLY_ONCE_BETA.equals(getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG)))
 {
+            return StreamThread.ProcessingMode.EXACTLY_ONCE_V2;
+        } else if 
(EXACTLY_ONCE_V2.equals(getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG))) {
+            return StreamThread.ProcessingMode.EXACTLY_ONCE_V2;
+        } else {
+            return StreamThread.ProcessingMode.AT_LEAST_ONCE;
         }
     }
 
+    public boolean eosEnabled() {

Review comment:
       This is a public API change -- to avoid this (ie also "leaks" and users 
won't need this method) we originally added it to `StreamThread` and not to 
`StreamsConfig` (even if it might be "cleaner" to have it on `StreamsConfig`)...
   
   As an alternative, we would introduce `class InternalStreamsConfig extends 
StreamsConfig`, add the method their.... (If we do this, might be worth to 
split out a refactoring PR and not piggy-back on the KIP PR...)
   
   Same for the other method below.

##########
File path: docs/streams/upgrade-guide.html
##########
@@ -53,17 +53,19 @@ <h1>Upgrade Guide and API Changes</h1>
     </ul>
 
     <p>
-        Starting in Kafka Streams 2.6.x, a new processing mode is available, 
named EOS version 2, which is configurable by setting 
-        <code>processing.guarantee</code> to <code>"exactly_once_beta"</code>.
-        <b>NOTE:</b> The <code>"exactly_once_beta"</code> processing mode is 
ready for production (<i>i.e.</i>, it's not "beta" software). 
+        Starting in Kafka Streams 2.6.x, a new processing mode is available, 
named EOS version 2. This can be configured
+        by setting <code>StreamsConfig.PROCESSING_GUARANTEE</code> to 
<code>StreamsConfig.EXACTLY_ONCE_V2</code> for
+        application versions 3.0+, or setting it to 
<code>StreamsConfig.EXACTLY_ONCE_BETA</code> for versions between 2.6 and 3.0.
         To use this new feature, your brokers must be on version 2.5.x or 
newer.
-        A switch from <code>"exactly_once"</code> to 
<code>"exactly_once_beta"</code> (or the other way around) is
-        only possible if the application is on version 2.6.x.
-        If you want to upgrade your application from an older version and 
enable this feature,
-        you first need to upgrade your application to version 2.6.x, staying 
on <code>"exactly_once"</code>,
-        and then do second round of rolling bounces to switch to 
<code>"exactly_once_beta"</code>.
-        For a downgrade, do the reverse: first switch the config from 
<code>"exactly_once_beta"</code> to
-        <code>"exactly_once"</code> to disable the feature in your 2.6.x 
application.
+        If you want to upgrade your EOS application from an older version and 
enable this feature,
+        you first need to upgrade your application to version 2.6.x, staying 
on <code>StreamsConfig.EXACTLY_ONCE</code>,
+        and then do second round of rolling bounces to switch to 
<code>StreamsConfig.EXACTLY_ONCE_BETA</code>. If you

Review comment:
       `EXACTLY_ONCE_BETA` or `EXACTLY_ONCE_V2`?

##########
File path: docs/streams/upgrade-guide.html
##########
@@ -53,17 +53,19 @@ <h1>Upgrade Guide and API Changes</h1>
     </ul>
 
     <p>
-        Starting in Kafka Streams 2.6.x, a new processing mode is available, 
named EOS version 2, which is configurable by setting 
-        <code>processing.guarantee</code> to <code>"exactly_once_beta"</code>.
-        <b>NOTE:</b> The <code>"exactly_once_beta"</code> processing mode is 
ready for production (<i>i.e.</i>, it's not "beta" software). 
+        Starting in Kafka Streams 2.6.x, a new processing mode is available, 
named EOS version 2. This can be configured
+        by setting <code>StreamsConfig.PROCESSING_GUARANTEE</code> to 
<code>StreamsConfig.EXACTLY_ONCE_V2</code> for
+        application versions 3.0+, or setting it to 
<code>StreamsConfig.EXACTLY_ONCE_BETA</code> for versions between 2.6 and 3.0.

Review comment:
       `between 2.6 and 2.8` ?




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