Skip to content

Commit

Permalink
KAFKA-16327: Removed Deprecated variable StreamsConfig#TOPOLOGY_OPTIM…
Browse files Browse the repository at this point in the history
…IZATION (#16744)

Reviewers: Matthias J. Sax <[email protected]>
  • Loading branch information
abhi-ksolves authored Aug 30, 2024
1 parent 2b49594 commit c23b6b0
Show file tree
Hide file tree
Showing 2 changed files with 6 additions and 14 deletions.
10 changes: 5 additions & 5 deletions docs/streams/upgrade-guide.html
Original file line number Diff line number Diff line change
Expand Up @@ -127,10 +127,10 @@ <h3 class="anchor-heading"><a id="streams_notable_changes" class="anchor-link"><
In deprecated <code>KStreamBuilder</code> class, when a <code>KTable</code> is created from a source topic via <code>KStreamBuilder.table()</code>, its materialized state store
will reuse the source topic as its changelog topic for restoring, and will disable logging to avoid appending new updates to the source topic; in the <code>StreamsBuilder</code> class introduced in 1.0, this behavior was changed
accidentally: we still reuse the source topic as the changelog topic for restoring, but will also create a separate changelog topic to append the update records from source topic to. In the 2.0 release, we have fixed this issue and now users
can choose whether or not to reuse the source topic based on the <code>StreamsConfig#TOPOLOGY_OPTIMIZATION</code>: if you are upgrading from the old <code>KStreamBuilder</code> class and hence you need to change your code to use
can choose whether or not to reuse the source topic based on the <code>StreamsConfig#TOPOLOGY_OPTIMIZATION_CONFIG</code>: if you are upgrading from the old <code>KStreamBuilder</code> class and hence you need to change your code to use
the new <code>StreamsBuilder</code>, you should set this config value to <code>StreamsConfig#OPTIMIZE</code> to continue reusing the source topic; if you are upgrading from 1.0 or 1.1 where you are already using <code>StreamsBuilder</code> and hence have already
created a separate changelog topic, you should set this config value to <code>StreamsConfig#NO_OPTIMIZATION</code> when upgrading to {{fullDotVersion}} in order to use that changelog topic for restoring the state store.
More details about the new config <code>StreamsConfig#TOPOLOGY_OPTIMIZATION</code> can be found in <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-295%3A+Add+Streams+Configuration+Allowing+for+Optional+Topology+Optimization">KIP-295</a>.
More details about the new config <code>StreamsConfig#TOPOLOGY_OPTIMIZATION_CONFIG</code> can be found in <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-295%3A+Add+Streams+Configuration+Allowing+for+Optional+Topology+Optimization">KIP-295</a>.
</p>

<h3><a id="streams_api_changes_400" href="#streams_api_changes_400">Streams API changes in 4.0.0</a></h3>
Expand Down Expand Up @@ -928,7 +928,7 @@ <h3 class="anchor-heading"><a id="streams_api_changes_230" class="anchor-link"><

<p>
As of 2.3.0 we've updated how to turn on optimizations. Now to enable optimizations, you need to do two things.
First add this line to your properties <code>properties.setProperty(StreamsConfig.TOPOLOGY_OPTIMIZATION, StreamsConfig.OPTIMIZE);</code>, as you have done before.
First add this line to your properties <code>properties.setProperty(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, StreamsConfig.OPTIMIZE);</code>, as you have done before.
Second, when constructing your <code>KafkaStreams</code> instance, you'll need to pass your configuration properties when building your
topology by using the overloaded <code>StreamsBuilder.build(Properties)</code> method.
For example <code>KafkaStreams myStream = new KafkaStreams(streamsBuilder.build(properties), properties)</code>.
Expand Down Expand Up @@ -1085,9 +1085,9 @@ <h3 class="anchor-heading"><a id="streams_api_changes_210" class="anchor-link"><

<p>
We've added an overloaded <code>StreamsBuilder#build</code> method that accepts an instance of <code>java.util.Properties</code> with the intent of using the
<code>StreamsConfig#TOPOLOGY_OPTIMIZATION</code> config added in Kafka Streams 2.0. Before 2.1, when building a topology with
<code>StreamsConfig#TOPOLOGY_OPTIMIZATION_CONFIG</code> config added in Kafka Streams 2.0. Before 2.1, when building a topology with
the DSL, Kafka Streams writes the physical plan as the user makes calls on the DSL. Now by providing a <code>java.util.Properties</code> instance when
executing a <code>StreamsBuilder#build</code> call, Kafka Streams can optimize the physical plan of the topology, provided the <code>StreamsConfig#TOPOLOGY_OPTIMIZATION</code>
executing a <code>StreamsBuilder#build</code> call, Kafka Streams can optimize the physical plan of the topology, provided the <code>StreamsConfig#TOPOLOGY_OPTIMIZATION_CONFIG</code>
config is set to <code>StreamsConfig#OPTIMIZE</code>. By setting <code>StreamsConfig#OPTIMIZE</code> in addition to the <code>KTable</code> optimization of
reusing the source topic as the changelog topic, the topology may be optimized to merge redundant repartition topics into one
repartition topic. The original no parameter version of <code>StreamsBuilder#build</code> is still available for those who wish to not
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -818,14 +818,6 @@ public class StreamsConfig extends AbstractConfig {
public static final String WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG = "windowstore.changelog.additional.retention.ms";
private static final String WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_DOC = "Added to a windows maintainMs to ensure data is not deleted from the log prematurely. Allows for clock drift. Default is 1 day";

/**
* {@code topology.optimization}
* @deprecated since 2.7; use {@link #TOPOLOGY_OPTIMIZATION_CONFIG} instead
*/
@Deprecated
public static final String TOPOLOGY_OPTIMIZATION = TOPOLOGY_OPTIMIZATION_CONFIG;


private static final String[] NON_CONFIGURABLE_CONSUMER_DEFAULT_CONFIGS =
new String[] {ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, ConsumerConfig.GROUP_PROTOCOL_CONFIG};
private static final String[] NON_CONFIGURABLE_CONSUMER_EOS_CONFIGS =
Expand Down Expand Up @@ -1261,7 +1253,7 @@ public static class InternalConfig {
public static boolean stateUpdaterEnabled(final Map<String, Object> configs) {
return InternalConfig.getBoolean(configs, InternalConfig.STATE_UPDATER_ENABLED, true);
}

// Private API to enable processing threads (i.e. polling is decoupled from processing)
public static final String PROCESSING_THREADS_ENABLED = "__processing.threads.enabled__";

Expand Down

0 comments on commit c23b6b0

Please sign in to comment.