[ 
https://issues.apache.org/jira/browse/KAFKA-6935?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16504114#comment-16504114
 ] 

ASF GitHub Bot commented on KAFKA-6935:
---------------------------------------

guozhangwang closed pull request #5071: KAFKA-6935: Add config for allowing 
optional optimization
URL: https://github.com/apache/kafka/pull/5071
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

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 18dc891682d..05c064e8f27 100644
--- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
+++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
@@ -16,8 +16,8 @@
  */
 package org.apache.kafka.streams;
 
-import org.apache.kafka.clients.admin.AdminClient;
 import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.admin.AdminClient;
 import org.apache.kafka.clients.admin.AdminClientConfig;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.KafkaConsumer;
@@ -32,10 +32,10 @@
 import org.apache.kafka.common.metrics.Sensor;
 import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.streams.errors.DefaultProductionExceptionHandler;
 import org.apache.kafka.streams.errors.DeserializationExceptionHandler;
 import org.apache.kafka.streams.errors.LogAndFailExceptionHandler;
 import org.apache.kafka.streams.errors.ProductionExceptionHandler;
-import org.apache.kafka.streams.errors.DefaultProductionExceptionHandler;
 import org.apache.kafka.streams.errors.StreamsException;
 import org.apache.kafka.streams.processor.DefaultPartitionGrouper;
 import org.apache.kafka.streams.processor.FailOnInvalidTimestamp;
@@ -195,6 +195,16 @@
      */
     public static final String ADMIN_CLIENT_PREFIX = "admin.";
 
+    /**
+     * Config value for parameter (@link #TOPOLOGY_OPTIMIZATION 
"topology.optimization" for disabling topology optimization
+     */
+    public static final String NO_OPTIMIZATION = "none";
+
+    /**
+     * Config value for parameter (@link #TOPOLOGY_OPTIMIZATION 
"topology.optimization" for enabling topology optimization
+     */
+    public static final String OPTIMIZE = "all";
+
     /**
      * Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} 
for upgrading an application from version {@code 0.10.0.x}.
      */
@@ -382,6 +392,10 @@
     public static final String STATE_DIR_CONFIG = "state.dir";
     private static final String STATE_DIR_DOC = "Directory location for state 
store.";
 
+    /** {@code topology.optimization} */
+    public static final String TOPOLOGY_OPTIMIZATION = "topology.optimization";
+    private static final String TOPOLOGY_OPTIMIZATION_DOC = "A configuration 
telling Kafka Streams if it should optimize the topology, disabled by default";
+
     /** {@code upgrade.from} */
     public static final String UPGRADE_FROM_CONFIG = "upgrade.from";
     public static final String UPGRADE_FROM_DOC = "Allows upgrading from 
versions 0.10.0/0.10.1/0.10.2/0.11.0/1.0/1.1 to version 1.2 (or newer) in a 
backward compatible way. " +
@@ -480,6 +494,12 @@
                     CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL,
                     Importance.MEDIUM,
                     CommonClientConfigs.SECURITY_PROTOCOL_DOC)
+            .define(TOPOLOGY_OPTIMIZATION,
+                    Type.STRING,
+                    NO_OPTIMIZATION,
+                    in(NO_OPTIMIZATION, OPTIMIZE),
+                    Importance.MEDIUM,
+                    TOPOLOGY_OPTIMIZATION_DOC)
 
             // LOW
 
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java 
b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
index e991b6ff89e..cdd4d097ba3 100644
--- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
@@ -45,6 +45,7 @@
 import static org.apache.kafka.common.requests.IsolationLevel.READ_COMMITTED;
 import static org.apache.kafka.common.requests.IsolationLevel.READ_UNCOMMITTED;
 import static org.apache.kafka.streams.StreamsConfig.EXACTLY_ONCE;
+import static org.apache.kafka.streams.StreamsConfig.TOPOLOGY_OPTIMIZATION;
 import static org.apache.kafka.streams.StreamsConfig.adminClientPrefix;
 import static org.apache.kafka.streams.StreamsConfig.consumerPrefix;
 import static org.apache.kafka.streams.StreamsConfig.producerPrefix;
@@ -583,6 +584,27 @@ public void 
shouldThrowExceptionIfMaxInflightRequestsGreatherThanFiveIfEosEnable
         }
     }
 
+    @Test
+    public void shouldSpecifyNoOptimizationWhenNotExplicitlyAddedToConfigs() {
+        final String expectedOptimizeConfig = "none";
+        final String actualOptimizedConifig = 
streamsConfig.getString(TOPOLOGY_OPTIMIZATION);
+        assertEquals("Optimization should be \"none\"", 
expectedOptimizeConfig, actualOptimizedConifig);
+    }
+
+    @Test
+    public void shouldSpecifyOptimizationWhenNotExplicitlyAddedToConfigs() {
+        final String expectedOptimizeConfig = "all";
+        props.put(TOPOLOGY_OPTIMIZATION, "all");
+        final StreamsConfig config = new StreamsConfig(props);
+        final String actualOptimizedConifig = 
config.getString(TOPOLOGY_OPTIMIZATION);
+        assertEquals("Optimization should be \"all\"", expectedOptimizeConfig, 
actualOptimizedConifig);
+    }
+
+    @Test(expected = ConfigException.class)
+    public void 
shouldThrowConfigExceptionWhenOptimizationConfigNotValueInRange() {
+        props.put(TOPOLOGY_OPTIMIZATION, "maybe");
+        new StreamsConfig(props);
+    }
 
     static class MisconfiguredSerde implements Serde {
         @Override


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> KIP-295 Add Streams Config for Optional Optimization
> ----------------------------------------------------
>
>                 Key: KAFKA-6935
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6935
>             Project: Kafka
>          Issue Type: New Feature
>          Components: streams
>    Affects Versions: 2.0.0
>            Reporter: Bill Bejeck
>            Assignee: Bill Bejeck
>            Priority: Major
>              Labels: kip
>             Fix For: 2.0.0
>
>
> KIP-295: 
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-295%3A+Add+Streams+Configuration+Allowing+for+Optional+Topology+Optimization



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to