mirror of https://github.com/apache/kafka.git
KAFKA-16327: Removed Deprecated variable StreamsConfig#TOPOLOGY_OPTIMIZATION (#16744)
Reviewers: Matthias J. Sax <matthias@confluent.io>
This commit is contained in:
parent
2b495945a2
commit
c23b6b0365
|
@ -127,10 +127,10 @@
|
||||||
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
|
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
|
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
|
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
|
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.
|
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>
|
</p>
|
||||||
|
|
||||||
<h3><a id="streams_api_changes_400" href="#streams_api_changes_400">Streams API changes in 4.0.0</a></h3>
|
<h3><a id="streams_api_changes_400" href="#streams_api_changes_400">Streams API changes in 4.0.0</a></h3>
|
||||||
|
@ -928,7 +928,7 @@
|
||||||
|
|
||||||
<p>
|
<p>
|
||||||
As of 2.3.0 we've updated how to turn on optimizations. Now to enable optimizations, you need to do two things.
|
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
|
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.
|
topology by using the overloaded <code>StreamsBuilder.build(Properties)</code> method.
|
||||||
For example <code>KafkaStreams myStream = new KafkaStreams(streamsBuilder.build(properties), properties)</code>.
|
For example <code>KafkaStreams myStream = new KafkaStreams(streamsBuilder.build(properties), properties)</code>.
|
||||||
|
@ -1085,9 +1085,9 @@
|
||||||
|
|
||||||
<p>
|
<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
|
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
|
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
|
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
|
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
|
repartition topic. The original no parameter version of <code>StreamsBuilder#build</code> is still available for those who wish to not
|
||||||
|
|
|
@ -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";
|
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";
|
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 =
|
private static final String[] NON_CONFIGURABLE_CONSUMER_DEFAULT_CONFIGS =
|
||||||
new String[] {ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, ConsumerConfig.GROUP_PROTOCOL_CONFIG};
|
new String[] {ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, ConsumerConfig.GROUP_PROTOCOL_CONFIG};
|
||||||
private static final String[] NON_CONFIGURABLE_CONSUMER_EOS_CONFIGS =
|
private static final String[] NON_CONFIGURABLE_CONSUMER_EOS_CONFIGS =
|
||||||
|
|
Loading…
Reference in New Issue