This is an automated email from the ASF dual-hosted git repository.

mjsax pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 4e76cc19cdc KAFKA-8416: Clarify two-step requirement for enabling 
Streams topology optimizations (#21556)
4e76cc19cdc is described below

commit 4e76cc19cdc0f698e33f98a8426d2214e1bc6493
Author: Emmanuel Oppong <[email protected]>
AuthorDate: Fri Feb 27 13:01:31 2026 -0600

    KAFKA-8416: Clarify two-step requirement for enabling Streams topology 
optimizations (#21556)
    
    The existing docs mentioned both steps in a single dense sentence,
    making it easy to miss that Properties must also be passed to
    StreamsBuilder.build().
    
    Added a numbered list and code example clearly showing both steps,  with
    an explicit warning that calling build() without Properties  silently
    skips optimizations even when the config is set.
    
    Reviewers: Matthias J. Sax <[email protected]>
---
 docs/streams/developer-guide/config-streams.md        | 19 ++++++++++++++++++-
 .../java/org/apache/kafka/streams/StreamsBuilder.java | 14 +++++++++++++-
 .../java/org/apache/kafka/streams/StreamsConfig.java  | 14 ++++++++++++--
 3 files changed, 43 insertions(+), 4 deletions(-)

diff --git a/docs/streams/developer-guide/config-streams.md 
b/docs/streams/developer-guide/config-streams.md
index 0528982fce1..12f21a8e325 100644
--- a/docs/streams/developer-guide/config-streams.md
+++ b/docs/streams/developer-guide/config-streams.md
@@ -1566,7 +1566,24 @@ We recommend listing specific optimizations in the 
config for production code so
 
 These optimizations include moving/reducing repartition topics and reusing the 
source topic as the changelog for source KTables. These optimizations will save 
on network traffic and storage in Kafka without changing the semantics of your 
applications. Enabling them is recommended. 
 
-Note that you need to do two things to enable optimizations. In addition to 
setting this config to `StreamsConfig.OPTIMIZE`, you'll need to pass in your 
configuration properties when building your topology by using the overloaded 
`StreamsBuilder.build(Properties)` method. For example `KafkaStreams myStream = 
new KafkaStreams(streamsBuilder.build(properties), properties)`. 
+**Important:** Enabling optimizations requires two steps. Both are necessary — 
setting the config alone is not enough:
+
+1. Set `topology.optimization` to `StreamsConfig.OPTIMIZE` (or a 
comma-separated list of specific optimizations) in your `Properties` object.
+2. Pass the same `Properties` object to the overloaded 
`StreamsBuilder.build(Properties)` method when building your topology.
+
+For example:
+
+```java
+Properties properties = new Properties();
+properties.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, 
StreamsConfig.OPTIMIZE);
+
+// Step 2: pass properties to build() — this is required for optimizations to 
take effect
+Topology topology = streamsBuilder.build(properties);
+
+KafkaStreams myStream = new KafkaStreams(topology, properties);
+```
+
+If you call `streamsBuilder.build()` without passing the `Properties` object, 
optimizations will **not** be applied even if the config is set. 
  
  #### upgrade.from
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java 
b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java
index 2dbf48d67a4..a7483bc1452 100644
--- a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java
+++ b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java
@@ -577,16 +577,28 @@ public class StreamsBuilder {
     /**
      * Returns the {@link Topology} that represents the specified processing 
logic.
      * Note that using this method means no optimizations are performed.
+     * To enable topology optimizations, use {@link #build(Properties)} 
instead and pass
+     * a {@link Properties} object with {@code topology.optimization} set to
+     * {@link StreamsConfig#OPTIMIZE}.
      *
      * @return the {@link Topology} that represents the specified processing 
logic
      */
     public synchronized Topology build() {
         return build(null);
     }
-    
+
     /**
      * Returns the {@link Topology} that represents the specified processing 
logic and accepts
      * a {@link Properties} instance used to indicate whether to optimize 
topology or not.
+     * <p>
+     * To enable topology optimizations, two steps are required:
+     * <ol>
+     *   <li>Set {@code topology.optimization} to {@link 
StreamsConfig#OPTIMIZE} in the
+     *       provided {@link Properties}.</li>
+     *   <li>Pass that same {@link Properties} object to this method.</li>
+     * </ol>
+     * If {@code topology.optimization} is not set to {@link 
StreamsConfig#OPTIMIZE} in
+     * {@code props}, no optimizations will be applied even if this method is 
called.
      *
      * @param props the {@link Properties} used for building possibly 
optimized topology
      * @return the {@link Topology} that represents the specified processing 
logic
diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java 
b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
index 91f34111abd..bc715ca8175 100644
--- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
+++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
@@ -799,7 +799,15 @@ public class StreamsConfig extends AbstractConfig {
         "Note that when upgrading from 3.5 to a newer version it is never 
required to specify this config, " +
         "while upgrading live directly to 4.0+ from 2.3 or below is no longer 
supported even with this config.";
 
-    /** {@code topology.optimization} */
+    /**
+     * {@code topology.optimization}
+     *
+     * <p><b>Important:</b> Setting this config alone is not sufficient to 
enable optimizations.
+     * You must also pass the {@link java.util.Properties} object to
+     * {@link 
org.apache.kafka.streams.StreamsBuilder#build(java.util.Properties)} when 
building
+     * your topology. Calling {@code StreamsBuilder.build()} without the 
properties argument will
+     * result in no optimizations being applied, even if this config is set.
+     */
     public static final String TOPOLOGY_OPTIMIZATION_CONFIG = 
"topology.optimization";
     private static final String CONFIG_ERROR_MSG = "Acceptable values are:"
         + " \"+NO_OPTIMIZATION+\", \"+OPTIMIZE+\", "
@@ -809,7 +817,9 @@ public class StreamsConfig extends AbstractConfig {
     private static final String TOPOLOGY_OPTIMIZATION_DOC = "A configuration 
telling Kafka "
         + "Streams if it should optimize the topology and what optimizations 
to apply. "
         + CONFIG_ERROR_MSG
-        + "\"NO_OPTIMIZATION\" by default.";
+        + "\"NO_OPTIMIZATION\" by default. "
+        + "Note: this config must also be passed to 
StreamsBuilder#build(Properties) "
+        + "for optimizations to take effect.";
 
     /**
      * {@code windowed.inner.class.serde}

Reply via email to