diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh
index bb786da43ca82..4dd092323b043 100755
--- a/bin/kafka-run-class.sh
+++ b/bin/kafka-run-class.sh
@@ -69,28 +69,50 @@ do
fi
done
-for file in "$base_dir"/clients/build/libs/kafka-clients*.jar;
-do
- if should_include_file "$file"; then
- CLASSPATH="$CLASSPATH":"$file"
- fi
-done
+if [ -z "$UPGRADE_KAFKA_STREAMS_TEST_VERSION" ]; then
+ clients_lib_dir=$(dirname $0)/../clients/build/libs
+ streams_lib_dir=$(dirname $0)/../streams/build/libs
+ rocksdb_lib_dir=$(dirname $0)/../streams/build/dependant-libs-${SCALA_VERSION}
+else
+ clients_lib_dir=/opt/kafka-$UPGRADE_KAFKA_STREAMS_TEST_VERSION/libs
+ streams_lib_dir=$clients_lib_dir
+ rocksdb_lib_dir=$streams_lib_dir
+fi
+
-for file in "$base_dir"/streams/build/libs/kafka-streams*.jar;
+for file in "$clients_lib_dir"/kafka-clients*.jar;
do
if should_include_file "$file"; then
CLASSPATH="$CLASSPATH":"$file"
fi
done
-for file in "$base_dir"/streams/examples/build/libs/kafka-streams-examples*.jar;
+for file in "$streams_lib_dir"/kafka-streams*.jar;
do
if should_include_file "$file"; then
CLASSPATH="$CLASSPATH":"$file"
fi
done
-for file in "$base_dir"/streams/build/dependant-libs-${SCALA_VERSION}/rocksdb*.jar;
+if [ -z "$UPGRADE_KAFKA_STREAMS_TEST_VERSION" ]; then
+ for file in "$base_dir"/streams/examples/build/libs/kafka-streams-examples*.jar;
+ do
+ if should_include_file "$file"; then
+ CLASSPATH="$CLASSPATH":"$file"
+ fi
+ done
+else
+ VERSION_NO_DOTS=`echo $UPGRADE_KAFKA_STREAMS_TEST_VERSION | sed 's/\.//g'`
+ SHORT_VERSION_NO_DOTS=${VERSION_NO_DOTS:0:((${#VERSION_NO_DOTS} - 1))} # remove last char, ie, bug-fix number
+ for file in "$base_dir"/streams/upgrade-system-tests-$SHORT_VERSION_NO_DOTS/build/libs/kafka-streams-upgrade-system-tests*.jar;
+ do
+ if should_include_file "$file"; then
+ CLASSPATH="$CLASSPATH":"$file"
+ fi
+ done
+fi
+
+for file in "$rocksdb_lib_dir"/rocksdb*.jar;
do
CLASSPATH="$CLASSPATH":"$file"
done
diff --git a/build.gradle b/build.gradle
index 5e4c35643c2a1..f4d1fb3e090f4 100644
--- a/build.gradle
+++ b/build.gradle
@@ -1019,6 +1019,66 @@ project(':streams:examples') {
}
}
+project(':streams:upgrade-system-tests-0100') {
+ archivesBaseName = "kafka-streams-upgrade-system-tests-0100"
+
+ dependencies {
+ testCompile libs.kafkaStreams_0100
+ }
+
+ systemTestLibs {
+ dependsOn testJar
+ }
+}
+
+project(':streams:upgrade-system-tests-0101') {
+ archivesBaseName = "kafka-streams-upgrade-system-tests-0101"
+
+ dependencies {
+ testCompile libs.kafkaStreams_0101
+ }
+
+ systemTestLibs {
+ dependsOn testJar
+ }
+}
+
+project(':streams:upgrade-system-tests-0102') {
+ archivesBaseName = "kafka-streams-upgrade-system-tests-0102"
+
+ dependencies {
+ testCompile libs.kafkaStreams_0102
+ }
+
+ systemTestLibs {
+ dependsOn testJar
+ }
+}
+
+project(':streams:upgrade-system-tests-0110') {
+ archivesBaseName = "kafka-streams-upgrade-system-tests-0110"
+
+ dependencies {
+ testCompile libs.kafkaStreams_0110
+ }
+
+ systemTestLibs {
+ dependsOn testJar
+ }
+}
+
+project(':streams:upgrade-system-tests-10') {
+ archivesBaseName = "kafka-streams-upgrade-system-tests-10"
+
+ dependencies {
+ testCompile libs.kafkaStreams_10
+ }
+
+ systemTestLibs {
+ dependsOn testJar
+ }
+}
+
project(':jmh-benchmarks') {
apply plugin: 'com.github.johnrengelman.shadow'
diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml
index f23805e665f93..1ec535f6f32cf 100644
--- a/checkstyle/suppressions.xml
+++ b/checkstyle/suppressions.xml
@@ -189,7 +189,7 @@
files="SmokeTestDriver.java"/>
+ files="KStreamKStreamJoinTest.java|SmokeTestDriver.java"/>
diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientCallbackHandler.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientCallbackHandler.java
index 31c51c22ca38c..4c17e596b3206 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientCallbackHandler.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientCallbackHandler.java
@@ -16,7 +16,9 @@
*/
package org.apache.kafka.common.security.authenticator;
-import java.util.Map;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.network.Mode;
+import org.apache.kafka.common.security.scram.ScramExtensionsCallback;
import javax.security.auth.Subject;
import javax.security.auth.callback.Callback;
@@ -25,10 +27,7 @@
import javax.security.auth.callback.UnsupportedCallbackException;
import javax.security.sasl.AuthorizeCallback;
import javax.security.sasl.RealmCallback;
-
-import org.apache.kafka.common.config.SaslConfigs;
-import org.apache.kafka.common.network.Mode;
-import org.apache.kafka.common.security.scram.ScramExtensionsCallback;
+import java.util.Map;
/**
* Callback handler for Sasl clients. The callbacks required for the SASL mechanism
diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html
index b9b4a4e48cfd5..09557fa2c63e0 100644
--- a/docs/streams/upgrade-guide.html
+++ b/docs/streams/upgrade-guide.html
@@ -34,7 +34,7 @@
Upgrade Guide and API Changes
- If you want to upgrade from 1.0.x to 1.1.0 and you have customized window store implementations on the ReadOnlyWindowStore interface
+ If you want to upgrade from 1.0.x to 1.1.x and you have customized window store implementations on the ReadOnlyWindowStore interface
you'd need to update your code to incorporate the newly added public APIs.
Otherwise, if you are using Java 7 you don't need to make any code changes as the public API is fully backward compatible;
but if you are using Java 8 method references in your Kafka Streams code you might need to update your code to resolve method ambiguities.
@@ -43,28 +43,52 @@
Upgrade Guide and API Changes
- If you want to upgrade from 0.11.0.x to 1.0.0 you don't need to make any code changes as the public API is fully backward compatible.
+ If you want to upgrade from 0.10.2.x or 0.11.0.x to 1.1.x and you have customized window store implementations on the ReadOnlyWindowStore interface
+ you'd need to update your code to incorporate the newly added public APIs.
+ Otherwise, if you are using Java 7 you don't need to do any code changes as the public API is fully backward compatible;
+ but if you are using Java 8 method references in your Kafka Streams code you might need to update your code to resolve method ambiguities.
However, some public APIs were deprecated and thus it is recommended to update your code eventually to allow for future upgrades.
- See below for a complete list of 1.0.0 API and semantic changes that allow you to advance your application and/or simplify your code base.
-
-
-
- If you want to upgrade from 0.10.2.x to 0.11.0 you don't need to make any code changes as the public API is fully backward compatible.
- However, some configuration parameters were deprecated and thus it is recommended to update your code eventually to allow for future upgrades.
- See below for a complete list of 0.11.0 API and semantic changes that allow you to advance your application and/or simplify your code base.
+ See below a complete list of 1.1, 1.0,
+ and 0.11.0 API
+ and semantic changes that allow you to advance your application and/or simplify your code base, including the usage of new features.
+ Additionally, Streams API 1.1.x requires broker on-disk message format version 0.10 or higher; thus, you need to make sure that the message
+ format is configured correctly before you upgrade your Kafka Streams application.
- If you want to upgrade from 0.10.1.x to 0.10.2, see the Upgrade Section for 0.10.2.
- It highlights incompatible changes you need to consider to upgrade your code and application.
- See below for a complete list of 0.10.2 API and semantic changes that allow you to advance your application and/or simplify your code base.
+ If you want to upgrade from 0.10.1.x to 1.1.x see the Upgrade Sections for 0.10.2,
+ 0.11.0, and
+ 1.0, and
+ 1.1.
+ Note, that a brokers on-disk message format must be on version 0.10 or higher to run a Kafka Streams application version 1.1 or higher.
+ See below a complete list of 0.10.2, 0.11.0,
+ 1.0, and 1.1 API and semantical changes
+ that allow you to advance your application and/or simplify your code base, including the usage of new features.
- If you want to upgrade from 0.10.0.x to 0.10.1, see the Upgrade Section for 0.10.1.
- It highlights incompatible changes you need to consider to upgrade your code and application.
- See below a complete list of 0.10.1 API changes that allow you to advance your application and/or simplify your code base, including the usage of new features.
+ Upgrading from 0.10.0.x to 1.1.x directly is also possible.
+ Note, that a brokers must be on version 0.10.1 or higher and on-disk message format must be on version 0.10 or higher
+ to run a Kafka Streams application version 1.1 or higher.
+ See Streams API changes in 0.10.1, Streams API changes in 0.10.2,
+ Streams API changes in 0.11.0, and Streams API changes in 1.0
+ for a complete list of API changes.
+ Upgrading to 1.1.1 requires two rolling bounces with config upgrade.from="0.10.0" set for first upgrade phase
+ (cf. KIP-268).
+ As an alternative, an offline upgrade is also possible.
+
+
prepare your application instances for a rolling bounce and make sure that config upgrade.from is set to "0.10.0" for new version 1.1.1
+
bounce each instance of your application once
+
prepare your newly deployed 1.1.1 application instances for a second round of rolling bounces; make sure to remove the value for config upgrade.mode
+
bounce each instance of your application once more to complete the upgrade
+
+
Upgrading from 0.10.0.x to 1.1.0 requires an offline upgrade (rolling bounce upgrade is not supported)
+
+
stop all old (0.10.0.x) application instances
+
update your code and swap old code and jar file with new code and new jar file
Upgrading your Streams application from 0.11.0 to 1.0.0 does not require a broker upgrade.
- A Kafka Streams 1.0.0 application can connect to 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though).
- However, Kafka Streams 1.0 requires 0.10 message format or newer and does not work with older message formats.
+ A Kafka Streams 1.0.0 application can connect to 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though).
+ However, Kafka Streams 1.0 requires 0.10 message format or newer and does not work with older message formats.
If you are monitoring on streams metrics, you will need make some changes to the metrics names in your reporting and monitoring code, because the metrics sensor hierarchy was changed.
There are a few public APIs including ProcessorContext#schedule(), Processor#punctuate() and KStreamBuilder, TopologyBuilder are being deprecated by new APIs.
- We recommend making corresponding code changes, which should be very minor since the new APIs look quite similar, when you upgrade.
+ We recommend making corresponding code changes, which should be very minor since the new APIs look quite similar, when you upgrade.
Upgrading your Streams application from 0.10.2 to 1.0 does not require a broker upgrade.
+ A Kafka Streams 1.0 application can connect to 1.0, 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though).
+
If you are monitoring on streams metrics, you will need make some changes to the metrics names in your reporting and monitoring code, because the metrics sensor hierarchy was changed.
+
There are a few public APIs including ProcessorContext#schedule(), Processor#punctuate() and KStreamBuilder, TopologyBuilder are being deprecated by new APIs.
+ We recommend making corresponding code changes, which should be very minor since the new APIs look quite similar, when you upgrade.
+
If you specify customized key.serde, value.serde and timestamp.extractor in configs, it is recommended to use their replaced configure parameter as these configs are deprecated.
Upgrading your Streams application from 0.10.1 to 1.0 does not require a broker upgrade.
+ A Kafka Streams 1.0 application can connect to 1.0, 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though).
+
You need to recompile your code. Just swapping the Kafka Streams library jar file will not work and will break your application.
+
If you are monitoring on streams metrics, you will need make some changes to the metrics names in your reporting and monitoring code, because the metrics sensor hierarchy was changed.
+
There are a few public APIs including ProcessorContext#schedule(), Processor#punctuate() and KStreamBuilder, TopologyBuilder are being deprecated by new APIs.
+ We recommend making corresponding code changes, which should be very minor since the new APIs look quite similar, when you upgrade.
+
If you specify customized key.serde, value.serde and timestamp.extractor in configs, it is recommended to use their replaced configure parameter as these configs are deprecated.
+
If you use a custom (i.e., user implemented) timestamp extractor, you will need to update this code, because the TimestampExtractor interface was changed.
+
If you register custom metrics, you will need to update this code, because the StreamsMetric interface was changed.
Upgrading your Streams application from 0.10.0 to 1.0 does require a broker upgrade because a Kafka Streams 1.0 application can only connect to 0.1, 0.11.0, 0.10.2, or 0.10.1 brokers.
Upgrading from 0.10.0.x to 1.0.2 requires two rolling bounces with config upgrade.from="0.10.0" set for first upgrade phase
+ (cf. KIP-268).
+ As an alternative, an offline upgrade is also possible.
+
+
prepare your application instances for a rolling bounce and make sure that config upgrade.from is set to "0.10.0" for new version 0.11.0.3
+
bounce each instance of your application once
+
prepare your newly deployed 1.0.2 application instances for a second round of rolling bounces; make sure to remove the value for config upgrade.mode
+
bounce each instance of your application once more to complete the upgrade
+
+
+
Upgrading from 0.10.0.x to 1.0.0 or 1.0.1 requires an offline upgrade (rolling bounce upgrade is not supported)
+
+
stop all old (0.10.0.x) application instances
+
update your code and swap old code and jar file with new code and new jar file
+
restart all new (1.0.0 or 1.0.1) application instances
Kafka 0.11.0.0 introduces a new message format version as well as wire protocol changes. By following the recommended rolling upgrade plan below,
you guarantee no downtime during the upgrade. However, please review the notable changes in 0.11.0.0 before upgrading.
@@ -258,11 +324,55 @@
Upgrading your Streams application from 0.10.2 to 0.11.0 does not require a broker upgrade.
- A Kafka Streams 0.11.0 application can connect to 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though).
+ A Kafka Streams 0.11.0 application can connect to 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though).
If you specify customized key.serde, value.serde and timestamp.extractor in configs, it is recommended to use their replaced configure parameter as these configs are deprecated.
Upgrading your Streams application from 0.10.1 to 0.11.0 does not require a broker upgrade.
+ A Kafka Streams 0.11.0 application can connect to 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though).
+
You need to recompile your code. Just swapping the Kafka Streams library jar file will not work and will break your application.
+
If you specify customized key.serde, value.serde and timestamp.extractor in configs, it is recommended to use their replaced configure parameter as these configs are deprecated.
+
If you use a custom (i.e., user implemented) timestamp extractor, you will need to update this code, because the TimestampExtractor interface was changed.
+
If you register custom metrics, you will need to update this code, because the StreamsMetric interface was changed.
Upgrading your Streams application from 0.10.0 to 0.11.0 does require a broker upgrade because a Kafka Streams 0.11.0 application can only connect to 0.11.0, 0.10.2, or 0.10.1 brokers.
Upgrading from 0.10.0.x to 0.11.0.3 requires two rolling bounces with config upgrade.from="0.10.0" set for first upgrade phase
+ (cf. KIP-268).
+ As an alternative, an offline upgrade is also possible.
+
+
prepare your application instances for a rolling bounce and make sure that config upgrade.from is set to "0.10.0" for new version 0.11.0.3
+
bounce each instance of your application once
+
prepare your newly deployed 0.11.0.3 application instances for a second round of rolling bounces; make sure to remove the value for config upgrade.mode
+
bounce each instance of your application once more to complete the upgrade
+
+
+
Upgrading from 0.10.0.x to 0.11.0.0, 0.11.0.1, or 0.11.0.2 requires an offline upgrade (rolling bounce upgrade is not supported)
+
+
stop all old (0.10.0.x) application instances
+
update your code and swap old code and jar file with new code and new jar file
+
restart all new (0.11.0.0 , 0.11.0.1, or 0.11.0.2) application instances
Upgrading your Streams application from 0.10.0 to 0.10.2 does require a broker upgrade because a Kafka Streams 0.10.2 application can only connect to 0.10.2 or 0.10.1 brokers.
+
There are couple of API changes, that are not backward compatible (cf. Streams API changes in 0.10.2 for more details).
+ Thus, you need to update and recompile your code. Just swapping the Kafka Streams library jar file will not work and will break your application.
+
Upgrading from 0.10.0.x to 0.10.2.2 requires two rolling bounces with config upgrade.from="0.10.0" set for first upgrade phase
+ (cf. KIP-268).
+ As an alternative, an offline upgrade is also possible.
+
+
prepare your application instances for a rolling bounce and make sure that config upgrade.from is set to "0.10.0" for new version 0.10.2.2
+
bounce each instance of your application once
+
prepare your newly deployed 0.10.2.2 application instances for a second round of rolling bounces; make sure to remove the value for config upgrade.mode
+
bounce each instance of your application once more to complete the upgrade
+
+
+
Upgrading from 0.10.0.x to 0.10.2.0 or 0.10.2.1 requires an offline upgrade (rolling bounce upgrade is not supported)
+
+
stop all old (0.10.0.x) application instances
+
update your code and swap old code and jar file with new code and new jar file
+
restart all new (0.10.2.0 or 0.10.2.1) application instances
The default values for two configurations of the StreamsConfig class were changed to improve the resiliency of Kafka Streams applications. The internal Kafka Streams producer retries default value was changed from 0 to 10. The internal Kafka Streams consumer max.poll.interval.ms default value was changed from 300000 to Integer.MAX_VALUE.
@@ -491,6 +630,23 @@
There are couple of API changes, that are not backward compatible (cf. Streams API changes in 0.10.1 for more details).
Thus, you need to update and recompile your code. Just swapping the Kafka Streams library jar file will not work and will break your application.
+
Upgrading from 0.10.0.x to 0.10.1.2 requires two rolling bounces with config upgrade.from="0.10.0" set for first upgrade phase
+ (cf. KIP-268).
+ As an alternative, an offline upgrade is also possible.
+
+
prepare your application instances for a rolling bounce and make sure that config upgrade.from is set to "0.10.0" for new version 0.10.1.2
+
bounce each instance of your application once
+
prepare your newly deployed 0.10.1.2 application instances for a second round of rolling bounces; make sure to remove the value for config upgrade.mode
+
bounce each instance of your application once more to complete the upgrade
+
+
+
Upgrading from 0.10.0.x to 0.10.1.0 or 0.10.1.1 requires an offline upgrade (rolling bounce upgrade is not supported)
+
+
stop all old (0.10.0.x) application instances
+
update your code and swap old code and jar file with new code and new jar file
+
restart all new (0.10.1.0 or 0.10.1.1) application instances
0.10.0.0 has potential breaking changes (please review before upgrading) and possible performance impact following the upgrade. By following the recommended rolling upgrade plan below, you guarantee no downtime and no performance impact during and following the upgrade.
Note: Because new protocols are introduced, it is important to upgrade your Kafka clusters before upgrading your clients.
-
+
+
Notes to clients with version 0.9.0.0: Due to a bug introduced in 0.9.0.0,
clients that depend on ZooKeeper (old Scala high-level Consumer and MirrorMaker if used with the old consumer) will not
work with 0.10.0.x brokers. Therefore, 0.9.0.0 clients should be upgraded to 0.9.0.1 before brokers are upgraded to
0.10.0.x. This step is not necessary for 0.8.X or 0.9.0.1 clients.
+
For a rolling upgrade:
diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle
index b7a03dcd7528f..d82ec4b3d6a20 100644
--- a/gradle/dependencies.gradle
+++ b/gradle/dependencies.gradle
@@ -60,6 +60,11 @@ versions += [
scalaLogging: "3.7.2",
jopt: "5.0.4",
junit: "4.12",
+ kafka_0100: "0.10.0.1",
+ kafka_0101: "0.10.1.1",
+ kafka_0102: "0.10.2.1",
+ kafka_0110: "0.11.0.2",
+ kafka_10: "1.0.1",
lz4: "1.4",
metrics: "2.2.0",
// PowerMock 1.x doesn't support Java 9, so use PowerMock 2.0.0 beta
@@ -97,12 +102,16 @@ libs += [
jettyServlets: "org.eclipse.jetty:jetty-servlets:$versions.jetty",
jerseyContainerServlet: "org.glassfish.jersey.containers:jersey-container-servlet:$versions.jersey",
jmhCore: "org.openjdk.jmh:jmh-core:$versions.jmh",
- jmhGeneratorAnnProcess: "org.openjdk.jmh:jmh-generator-annprocess:$versions.jmh",
jmhCoreBenchmarks: "org.openjdk.jmh:jmh-core-benchmarks:$versions.jmh",
+ jmhGeneratorAnnProcess: "org.openjdk.jmh:jmh-generator-annprocess:$versions.jmh",
+ joptSimple: "net.sf.jopt-simple:jopt-simple:$versions.jopt",
junit: "junit:junit:$versions.junit",
+ kafkaStreams_0100: "org.apache.kafka:kafka-streams:$versions.kafka_0100",
+ kafkaStreams_0101: "org.apache.kafka:kafka-streams:$versions.kafka_0101",
+ kafkaStreams_0102: "org.apache.kafka:kafka-streams:$versions.kafka_0102",
+ kafkaStreams_0110: "org.apache.kafka:kafka-streams:$versions.kafka_0110",
+ kafkaStreams_10: "org.apache.kafka:kafka-streams:$versions.kafka_10",
log4j: "log4j:log4j:$versions.log4j",
- scalaLogging: "com.typesafe.scala-logging:scala-logging_$versions.baseScala:$versions.scalaLogging",
- joptSimple: "net.sf.jopt-simple:jopt-simple:$versions.jopt",
lz4: "org.lz4:lz4-java:$versions.lz4",
metrics: "com.yammer.metrics:metrics-core:$versions.metrics",
powermockJunit4: "org.powermock:powermock-module-junit4:$versions.powermock",
@@ -110,6 +119,7 @@ libs += [
reflections: "org.reflections:reflections:$versions.reflections",
rocksDBJni: "org.rocksdb:rocksdbjni:$versions.rocksDB",
scalaLibrary: "org.scala-lang:scala-library:$versions.scala",
+ scalaLogging: "com.typesafe.scala-logging:scala-logging_$versions.baseScala:$versions.scalaLogging",
scalaReflect: "org.scala-lang:scala-reflect:$versions.scala",
scalatest: "org.scalatest:scalatest_$versions.baseScala:$versions.scalatest",
scoveragePlugin: "org.scoverage:scalac-scoverage-plugin_$versions.baseScala:$versions.scoverage",
diff --git a/settings.gradle b/settings.gradle
index e599d01215cc9..03136849fd543 100644
--- a/settings.gradle
+++ b/settings.gradle
@@ -13,5 +13,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
-include 'core', 'examples', 'clients', 'tools', 'streams', 'streams:test-utils', 'streams:examples', 'log4j-appender',
+include 'core', 'examples', 'clients', 'tools', 'streams', 'streams:test-utils', 'streams:examples',
+ 'streams:upgrade-system-tests-0100', 'streams:upgrade-system-tests-0101', 'streams:upgrade-system-tests-0102',
+ 'streams:upgrade-system-tests-0110', 'streams:upgrade-system-tests-10', 'log4j-appender',
'connect:api', 'connect:transforms', 'connect:runtime', 'connect:json', 'connect:file', 'jmh-benchmarks'
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 ec0a1a8385b58..c1c50a64cbebe 100644
--- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
+++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
@@ -167,6 +167,11 @@ public class StreamsConfig extends AbstractConfig {
*/
public static final String ADMIN_CLIENT_PREFIX = "admin.";
+ /**
+ * Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 0.10.0.x}.
+ */
+ public static final String UPGRADE_FROM_0100 = "0.10.0";
+
/**
* Config value for parameter {@link #PROCESSING_GUARANTEE_CONFIG "processing.guarantee"} for at-least-once processing guarantees.
*/
@@ -326,6 +331,11 @@ public class StreamsConfig extends AbstractConfig {
public static final String TIMESTAMP_EXTRACTOR_CLASS_CONFIG = "timestamp.extractor";
private static final String TIMESTAMP_EXTRACTOR_CLASS_DOC = "Timestamp extractor class that implements the org.apache.kafka.streams.processor.TimestampExtractor interface. This config is deprecated, use " + DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG + " instead";
+ /** {@code upgrade.from} */
+ public static final String UPGRADE_FROM_CONFIG = "upgrade.from";
+ public static final String UPGRADE_FROM_DOC = "Allows upgrading from version 0.10.0 to version 0.10.1 (or newer) in a backward compatible way. " +
+ "Default is null. Accepted values are \"" + UPGRADE_FROM_0100 + "\" (for upgrading from 0.10.0.x).";
+
/**
* {@code value.serde}
* @deprecated Use {@link #DEFAULT_VALUE_SERDE_CLASS_CONFIG} instead.
@@ -548,6 +558,12 @@ public class StreamsConfig extends AbstractConfig {
10 * 60 * 1000,
Importance.LOW,
STATE_CLEANUP_DELAY_MS_DOC)
+ .define(UPGRADE_FROM_CONFIG,
+ ConfigDef.Type.STRING,
+ null,
+ in(null, UPGRADE_FROM_0100),
+ Importance.LOW,
+ UPGRADE_FROM_DOC)
.define(WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG,
Type.LONG,
24 * 60 * 60 * 1000,
@@ -779,6 +795,7 @@ public Map getConsumerConfigs(final String groupId,
consumerProps.put(CommonClientConfigs.CLIENT_ID_CONFIG, clientId + "-consumer");
// add configs required for stream partition assignor
+ consumerProps.put(UPGRADE_FROM_CONFIG, getString(UPGRADE_FROM_CONFIG));
consumerProps.put(REPLICATION_FACTOR_CONFIG, getInt(REPLICATION_FACTOR_CONFIG));
consumerProps.put(APPLICATION_SERVER_CONFIG, getString(APPLICATION_SERVER_CONFIG));
consumerProps.put(NUM_STANDBY_REPLICAS_CONFIG, getInt(NUM_STANDBY_REPLICAS_CONFIG));
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java
index 2a08308a2fd0f..1bd037681536d 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java
@@ -174,6 +174,8 @@ public int compare(TopicPartition p1, TopicPartition p2) {
private TaskManager taskManager;
private PartitionGrouper partitionGrouper;
+ private int userMetadataVersion = SubscriptionInfo.CURRENT_VERSION;
+
private InternalTopicManager internalTopicManager;
private CopartitionedTopicsValidator copartitionedTopicsValidator;
@@ -192,6 +194,12 @@ public void configure(final Map configs) {
final LogContext logContext = new LogContext(logPrefix);
log = logContext.logger(getClass());
+ final String upgradeMode = (String) configs.get(StreamsConfig.UPGRADE_FROM_CONFIG);
+ if (StreamsConfig.UPGRADE_FROM_0100.equals(upgradeMode)) {
+ log.info("Downgrading metadata version from 2 to 1 for upgrade from 0.10.0.x.");
+ userMetadataVersion = 1;
+ }
+
final Object o = configs.get(StreamsConfig.InternalConfig.TASK_MANAGER_FOR_PARTITION_ASSIGNOR);
if (o == null) {
KafkaException ex = new KafkaException("TaskManager is not specified");
@@ -249,7 +257,7 @@ public Subscription subscription(Set topics) {
final Set previousActiveTasks = taskManager.prevActiveTaskIds();
final Set standbyTasks = taskManager.cachedTasksIds();
standbyTasks.removeAll(previousActiveTasks);
- final SubscriptionInfo data = new SubscriptionInfo(taskManager.processId(), previousActiveTasks, standbyTasks, this.userEndPoint);
+ final SubscriptionInfo data = new SubscriptionInfo(userMetadataVersion, taskManager.processId(), previousActiveTasks, standbyTasks, this.userEndPoint);
taskManager.updateSubscriptionsFromMetadata(topics);
@@ -281,11 +289,16 @@ public Map assign(Cluster metadata, Map clientsMetadata = new HashMap<>();
+ int minUserMetadataVersion = SubscriptionInfo.CURRENT_VERSION;
for (Map.Entry entry : subscriptions.entrySet()) {
String consumerId = entry.getKey();
Subscription subscription = entry.getValue();
SubscriptionInfo info = SubscriptionInfo.decode(subscription.userData());
+ final int usedVersion = info.version;
+ if (usedVersion < minUserMetadataVersion) {
+ minUserMetadataVersion = usedVersion;
+ }
// create the new client metadata if necessary
ClientMetadata clientMetadata = clientsMetadata.get(info.processId);
@@ -546,7 +559,7 @@ public Map assign(Cluster metadata, Map activeTasks, Map>
this(CURRENT_VERSION, activeTasks, standbyTasks, hostState);
}
- protected AssignmentInfo(int version, List activeTasks, Map> standbyTasks,
+ public AssignmentInfo(int version, List activeTasks, Map> standbyTasks,
Map> hostState) {
this.version = version;
this.activeTasks = activeTasks;
@@ -153,8 +153,7 @@ public static AssignmentInfo decode(ByteBuffer data) {
}
}
- return new AssignmentInfo(activeTasks, standbyTasks, hostStateToTopicPartitions);
-
+ return new AssignmentInfo(version, activeTasks, standbyTasks, hostStateToTopicPartitions);
} catch (IOException ex) {
throw new TaskAssignmentException("Failed to decode AssignmentInfo", ex);
}
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java
index f583dbafc94f1..00227e799b848 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java
@@ -31,7 +31,7 @@ public class SubscriptionInfo {
private static final Logger log = LoggerFactory.getLogger(SubscriptionInfo.class);
- private static final int CURRENT_VERSION = 2;
+ public static final int CURRENT_VERSION = 2;
public final int version;
public final UUID processId;
@@ -43,7 +43,7 @@ public SubscriptionInfo(UUID processId, Set prevTasks, Set stand
this(CURRENT_VERSION, processId, prevTasks, standbyTasks, userEndPoint);
}
- private SubscriptionInfo(int version, UUID processId, Set prevTasks, Set standbyTasks, String userEndPoint) {
+ public SubscriptionInfo(int version, UUID processId, Set prevTasks, Set standbyTasks, String userEndPoint) {
this.version = version;
this.processId = processId;
this.prevTasks = prevTasks;
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 cc072d5508d99..a873e4c492022 100644
--- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
@@ -472,6 +472,7 @@ public void shouldNotOverrideUserConfigCommitIntervalMsIfExactlyOnceEnabled() {
assertThat(streamsConfig.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG), equalTo(commitIntervalMs));
}
+ @SuppressWarnings("deprecation")
@Test
public void shouldBeBackwardsCompatibleWithDeprecatedConfigs() {
final Properties props = minimalStreamsConfig();
@@ -506,6 +507,7 @@ public void shouldUseCorrectDefaultsWhenNoneSpecified() {
assertTrue(config.defaultTimestampExtractor() instanceof FailOnInvalidTimestamp);
}
+ @SuppressWarnings("deprecation")
@Test
public void shouldSpecifyCorrectKeySerdeClassOnErrorUsingDeprecatedConfigs() {
final Properties props = minimalStreamsConfig();
@@ -519,6 +521,7 @@ public void shouldSpecifyCorrectKeySerdeClassOnErrorUsingDeprecatedConfigs() {
}
}
+ @SuppressWarnings("deprecation")
@Test
public void shouldSpecifyCorrectKeySerdeClassOnError() {
final Properties props = minimalStreamsConfig();
@@ -532,6 +535,7 @@ public void shouldSpecifyCorrectKeySerdeClassOnError() {
}
}
+ @SuppressWarnings("deprecation")
@Test
public void shouldSpecifyCorrectValueSerdeClassOnErrorUsingDeprecatedConfigs() {
final Properties props = minimalStreamsConfig();
@@ -545,6 +549,7 @@ public void shouldSpecifyCorrectValueSerdeClassOnErrorUsingDeprecatedConfigs() {
}
}
+ @SuppressWarnings("deprecation")
@Test
public void shouldSpecifyCorrectValueSerdeClassOnError() {
final Properties props = minimalStreamsConfig();
@@ -567,9 +572,7 @@ public void configure(final Map configs, final boolean isKey) {
}
@Override
- public void close() {
-
- }
+ public void close() {}
@Override
public Serializer serializer() {
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java
index 4c12bb935448f..44e139a28bdbf 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java
@@ -313,6 +313,4 @@ private List> receiveMessages(final Deserializer
}
-
-
}
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java
index 02ab803735aa7..e4b07ba3852ee 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java
@@ -131,7 +131,7 @@ private void configurePartitionAssignor(final Map props) {
private void mockTaskManager(final Set prevTasks,
final Set cachedTasks,
final UUID processId,
- final InternalTopologyBuilder builder) throws NoSuchFieldException, IllegalAccessException {
+ final InternalTopologyBuilder builder) {
EasyMock.expect(taskManager.builder()).andReturn(builder).anyTimes();
EasyMock.expect(taskManager.prevActiveTaskIds()).andReturn(prevTasks).anyTimes();
EasyMock.expect(taskManager.cachedTasksIds()).andReturn(cachedTasks).anyTimes();
@@ -167,7 +167,7 @@ public void shouldInterleaveTasksByGroupId() {
}
@Test
- public void testSubscription() throws Exception {
+ public void testSubscription() {
builder.addSource(null, "source1", null, null, null, "topic1");
builder.addSource(null, "source2", null, null, null, "topic2");
builder.addProcessor("processor", new MockProcessorSupplier(), "source1", "source2");
@@ -195,7 +195,7 @@ public void testSubscription() throws Exception {
}
@Test
- public void testAssignBasic() throws Exception {
+ public void testAssignBasic() {
builder.addSource(null, "source1", null, null, null, "topic1");
builder.addSource(null, "source2", null, null, null, "topic2");
builder.addProcessor("processor", new MockProcessorSupplier(), "source1", "source2");
@@ -235,11 +235,9 @@ public void testAssignBasic() throws Exception {
// check assignment info
- Set allActiveTasks = new HashSet<>();
-
// the first consumer
AssignmentInfo info10 = checkAssignment(allTopics, assignments.get("consumer10"));
- allActiveTasks.addAll(info10.activeTasks);
+ Set allActiveTasks = new HashSet<>(info10.activeTasks);
// the second consumer
AssignmentInfo info11 = checkAssignment(allTopics, assignments.get("consumer11"));
@@ -259,7 +257,7 @@ public void testAssignBasic() throws Exception {
}
@Test
- public void shouldAssignEvenlyAcrossConsumersOneClientMultipleThreads() throws Exception {
+ public void shouldAssignEvenlyAcrossConsumersOneClientMultipleThreads() {
builder.addSource(null, "source1", null, null, null, "topic1");
builder.addSource(null, "source2", null, null, null, "topic2");
builder.addProcessor("processor", new MockProcessorSupplier(), "source1");
@@ -327,7 +325,7 @@ public void shouldAssignEvenlyAcrossConsumersOneClientMultipleThreads() throws E
}
@Test
- public void testAssignWithPartialTopology() throws Exception {
+ public void testAssignWithPartialTopology() {
builder.addSource(null, "source1", null, null, null, "topic1");
builder.addProcessor("processor1", new MockProcessorSupplier(), "source1");
builder.addStateStore(new MockStateStoreSupplier("store1", false), "processor1");
@@ -352,9 +350,8 @@ public void testAssignWithPartialTopology() throws Exception {
Map assignments = partitionAssignor.assign(metadata, subscriptions);
// check assignment info
- Set allActiveTasks = new HashSet<>();
AssignmentInfo info10 = checkAssignment(Utils.mkSet("topic1"), assignments.get("consumer10"));
- allActiveTasks.addAll(info10.activeTasks);
+ Set allActiveTasks = new HashSet<>(info10.activeTasks);
assertEquals(3, allActiveTasks.size());
assertEquals(allTasks, new HashSet<>(allActiveTasks));
@@ -362,7 +359,7 @@ public void testAssignWithPartialTopology() throws Exception {
@Test
- public void testAssignEmptyMetadata() throws Exception {
+ public void testAssignEmptyMetadata() {
builder.addSource(null, "source1", null, null, null, "topic1");
builder.addSource(null, "source2", null, null, null, "topic2");
builder.addProcessor("processor", new MockProcessorSupplier(), "source1", "source2");
@@ -392,9 +389,8 @@ public void testAssignEmptyMetadata() throws Exception {
new HashSet<>(assignments.get("consumer10").partitions()));
// check assignment info
- Set allActiveTasks = new HashSet<>();
AssignmentInfo info10 = checkAssignment(Collections.emptySet(), assignments.get("consumer10"));
- allActiveTasks.addAll(info10.activeTasks);
+ Set allActiveTasks = new HashSet<>(info10.activeTasks);
assertEquals(0, allActiveTasks.size());
assertEquals(Collections.emptySet(), new HashSet<>(allActiveTasks));
@@ -417,7 +413,7 @@ public void testAssignEmptyMetadata() throws Exception {
}
@Test
- public void testAssignWithNewTasks() throws Exception {
+ public void testAssignWithNewTasks() {
builder.addSource(null, "source1", null, null, null, "topic1");
builder.addSource(null, "source2", null, null, null, "topic2");
builder.addSource(null, "source3", null, null, null, "topic3");
@@ -450,13 +446,9 @@ public void testAssignWithNewTasks() throws Exception {
// check assigned partitions: since there is no previous task for topic 3 it will be assigned randomly so we cannot check exact match
// also note that previously assigned partitions / tasks may not stay on the previous host since we may assign the new task first and
// then later ones will be re-assigned to other hosts due to load balancing
- Set allActiveTasks = new HashSet<>();
- Set allPartitions = new HashSet<>();
- AssignmentInfo info;
-
- info = AssignmentInfo.decode(assignments.get("consumer10").userData());
- allActiveTasks.addAll(info.activeTasks);
- allPartitions.addAll(assignments.get("consumer10").partitions());
+ AssignmentInfo info = AssignmentInfo.decode(assignments.get("consumer10").userData());
+ Set allActiveTasks = new HashSet<>(info.activeTasks);
+ Set allPartitions = new HashSet<>(assignments.get("consumer10").partitions());
info = AssignmentInfo.decode(assignments.get("consumer11").userData());
allActiveTasks.addAll(info.activeTasks);
@@ -471,7 +463,7 @@ public void testAssignWithNewTasks() throws Exception {
}
@Test
- public void testAssignWithStates() throws Exception {
+ public void testAssignWithStates() {
builder.setApplicationId(applicationId);
builder.addSource(null, "source1", null, null, null, "topic1");
builder.addSource(null, "source2", null, null, null, "topic2");
@@ -542,7 +534,10 @@ public void testAssignWithStates() throws Exception {
assertEquals(Utils.mkSet(task10, task11, task12), tasksForState(applicationId, "store3", tasks, topicGroups));
}
- private Set tasksForState(String applicationId, String storeName, List tasks, Map topicGroups) {
+ private Set tasksForState(final String applicationId,
+ final String storeName,
+ final List tasks,
+ final Map topicGroups) {
final String changelogTopic = ProcessorStateManager.storeChangelogTopic(applicationId, storeName);
Set ids = new HashSet<>();
@@ -560,7 +555,7 @@ private Set tasksForState(String applicationId, String storeName, List props = configProps();
props.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, "1");
StreamsConfig streamsConfig = new StreamsConfig(props);
@@ -598,13 +593,10 @@ public void testAssignWithStandbyReplicas() throws Exception {
Map assignments = partitionAssignor.assign(metadata, subscriptions);
- Set allActiveTasks = new HashSet<>();
- Set allStandbyTasks = new HashSet<>();
-
// the first consumer
AssignmentInfo info10 = checkAssignment(allTopics, assignments.get("consumer10"));
- allActiveTasks.addAll(info10.activeTasks);
- allStandbyTasks.addAll(info10.standbyTasks.keySet());
+ Set allActiveTasks = new HashSet<>(info10.activeTasks);
+ Set allStandbyTasks = new HashSet<>(info10.standbyTasks.keySet());
// the second consumer
AssignmentInfo info11 = checkAssignment(allTopics, assignments.get("consumer11"));
@@ -632,7 +624,7 @@ public void testAssignWithStandbyReplicas() throws Exception {
}
@Test
- public void testOnAssignment() throws Exception {
+ public void testOnAssignment() {
configurePartitionAssignor(Collections.emptyMap());
final List activeTaskList = Utils.mkList(task0, task3);
@@ -667,7 +659,7 @@ public void testOnAssignment() throws Exception {
}
@Test
- public void testAssignWithInternalTopics() throws Exception {
+ public void testAssignWithInternalTopics() {
builder.setApplicationId(applicationId);
builder.addInternalTopic("topicX");
builder.addSource(null, "source1", null, null, null, "topic1");
@@ -697,7 +689,7 @@ public void testAssignWithInternalTopics() throws Exception {
}
@Test
- public void testAssignWithInternalTopicThatsSourceIsAnotherInternalTopic() throws Exception {
+ public void testAssignWithInternalTopicThatsSourceIsAnotherInternalTopic() {
String applicationId = "test";
builder.setApplicationId(applicationId);
builder.addInternalTopic("topicX");
@@ -732,7 +724,7 @@ public void testAssignWithInternalTopicThatsSourceIsAnotherInternalTopic() throw
}
@Test
- public void shouldGenerateTasksForAllCreatedPartitions() throws Exception {
+ public void shouldGenerateTasksForAllCreatedPartitions() {
final StreamsBuilder builder = new StreamsBuilder();
final InternalTopologyBuilder internalTopologyBuilder = StreamsBuilderTest.internalTopologyBuilder(builder);
@@ -832,7 +824,7 @@ public Object apply(final Object value1, final Object value2) {
}
@Test
- public void shouldAddUserDefinedEndPointToSubscription() throws Exception {
+ public void shouldAddUserDefinedEndPointToSubscription() {
builder.setApplicationId(applicationId);
builder.addSource(null, "source", null, null, null, "input");
builder.addProcessor("processor", new MockProcessorSupplier(), "source");
@@ -851,7 +843,56 @@ public void shouldAddUserDefinedEndPointToSubscription() throws Exception {
}
@Test
- public void shouldMapUserEndPointToTopicPartitions() throws Exception {
+ public void shouldReturnLowestAssignmentVersionForDifferentSubscriptionVersions() {
+ final Map subscriptions = new HashMap<>();
+ final Set emptyTasks = Collections.emptySet();
+ subscriptions.put(
+ "consumer1",
+ new PartitionAssignor.Subscription(
+ Collections.singletonList("topic1"),
+ new SubscriptionInfo(1, UUID.randomUUID(), emptyTasks, emptyTasks, null).encode()
+ )
+ );
+ subscriptions.put(
+ "consumer2",
+ new PartitionAssignor.Subscription(
+ Collections.singletonList("topic1"),
+ new SubscriptionInfo(2, UUID.randomUUID(), emptyTasks, emptyTasks, null).encode()
+ )
+ );
+
+ mockTaskManager(
+ emptyTasks,
+ emptyTasks,
+ UUID.randomUUID(),
+ builder);
+ configurePartitionAssignor(Collections.emptyMap());
+
+ final Map assignment = partitionAssignor.assign(metadata, subscriptions);
+
+ assertEquals(2, assignment.size());
+ assertEquals(1, AssignmentInfo.decode(assignment.get("consumer1").userData()).version);
+ assertEquals(1, AssignmentInfo.decode(assignment.get("consumer2").userData()).version);
+ }
+
+ @Test
+ public void shouldDownGradeSubscription() {
+ final Set emptyTasks = Collections.emptySet();
+
+ mockTaskManager(
+ emptyTasks,
+ emptyTasks,
+ UUID.randomUUID(),
+ builder);
+ configurePartitionAssignor(Collections.singletonMap(StreamsConfig.UPGRADE_FROM_CONFIG, (Object) StreamsConfig.UPGRADE_FROM_0100));
+
+ PartitionAssignor.Subscription subscription = partitionAssignor.subscription(Utils.mkSet("topic1"));
+
+ assertEquals(1, SubscriptionInfo.decode(subscription.userData()).version);
+ }
+
+ @Test
+ public void shouldMapUserEndPointToTopicPartitions() {
builder.setApplicationId(applicationId);
builder.addSource(null, "source", null, null, null, "topic1");
builder.addProcessor("processor", new MockProcessorSupplier(), "source");
@@ -881,7 +922,7 @@ public void shouldMapUserEndPointToTopicPartitions() throws Exception {
}
@Test
- public void shouldThrowExceptionIfApplicationServerConfigIsNotHostPortPair() throws Exception {
+ public void shouldThrowExceptionIfApplicationServerConfigIsNotHostPortPair() {
builder.setApplicationId(applicationId);
mockTaskManager(Collections.emptySet(), Collections.emptySet(), UUID.randomUUID(), builder);
@@ -908,7 +949,7 @@ public void shouldThrowExceptionIfApplicationServerConfigPortIsNotAnInteger() {
}
@Test
- public void shouldNotLoopInfinitelyOnMissingMetadataAndShouldNotCreateRelatedTasks() throws Exception {
+ public void shouldNotLoopInfinitelyOnMissingMetadataAndShouldNotCreateRelatedTasks() {
final StreamsBuilder builder = new StreamsBuilder();
final InternalTopologyBuilder internalTopologyBuilder = StreamsBuilderTest.internalTopologyBuilder(builder);
@@ -1010,7 +1051,7 @@ public Object apply(final Object value1, final Object value2) {
}
@Test
- public void shouldUpdateClusterMetadataAndHostInfoOnAssignment() throws Exception {
+ public void shouldUpdateClusterMetadataAndHostInfoOnAssignment() {
final TopicPartition partitionOne = new TopicPartition("topic", 1);
final TopicPartition partitionTwo = new TopicPartition("topic", 2);
final Map> hostState = Collections.singletonMap(
@@ -1028,7 +1069,7 @@ public void shouldUpdateClusterMetadataAndHostInfoOnAssignment() throws Exceptio
}
@Test
- public void shouldNotAddStandbyTaskPartitionsToPartitionsForHost() throws Exception {
+ public void shouldNotAddStandbyTaskPartitionsToPartitionsForHost() {
final StreamsBuilder builder = new StreamsBuilder();
final InternalTopologyBuilder internalTopologyBuilder = StreamsBuilderTest.internalTopologyBuilder(builder);
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java
index ec94ad81acd6d..8032f7da86556 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java
@@ -64,10 +64,9 @@ public void shouldDecodePreviousVersion() throws IOException {
assertEquals(oldVersion.activeTasks, decoded.activeTasks);
assertEquals(oldVersion.standbyTasks, decoded.standbyTasks);
assertEquals(0, decoded.partitionsByHost.size()); // should be empty as wasn't in V1
- assertEquals(2, decoded.version); // automatically upgraded to v2 on decode;
+ assertEquals(1, decoded.version);
}
-
/**
* This is a clone of what the V1 encoding did. The encode method has changed for V2
* so it is impossible to test compatibility without having this
diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java
index 727c42175951a..9304cec59862b 100644
--- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java
+++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java
@@ -19,6 +19,7 @@
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.utils.Bytes;
import org.apache.kafka.streams.Consumed;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.StreamsBuilder;
@@ -30,10 +31,13 @@
import org.apache.kafka.streams.kstream.KTable;
import org.apache.kafka.streams.kstream.Materialized;
import org.apache.kafka.streams.kstream.Predicate;
+import org.apache.kafka.streams.kstream.Produced;
import org.apache.kafka.streams.kstream.Serialized;
import org.apache.kafka.streams.kstream.TimeWindows;
import org.apache.kafka.streams.kstream.ValueJoiner;
+import org.apache.kafka.streams.state.KeyValueStore;
import org.apache.kafka.streams.state.Stores;
+import org.apache.kafka.streams.state.WindowStore;
import java.io.File;
import java.util.Properties;
@@ -47,7 +51,7 @@ public class SmokeTestClient extends SmokeTestUtil {
private Thread thread;
private boolean uncaughtException = false;
- public SmokeTestClient(File stateDir, String kafka) {
+ public SmokeTestClient(final File stateDir, final String kafka) {
super();
this.stateDir = stateDir;
this.kafka = kafka;
@@ -57,7 +61,7 @@ public void start() {
streams = createKafkaStreams(stateDir, kafka);
streams.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
@Override
- public void uncaughtException(Thread t, Throwable e) {
+ public void uncaughtException(final Thread t, final Throwable e) {
System.out.println("SMOKE-TEST-CLIENT-EXCEPTION");
uncaughtException = true;
e.printStackTrace();
@@ -94,7 +98,7 @@ public void close() {
}
}
- private static KafkaStreams createKafkaStreams(File stateDir, String kafka) {
+ private static Properties getStreamsConfig(final File stateDir, final String kafka) {
final Properties props = new Properties();
props.put(StreamsConfig.APPLICATION_ID_CONFIG, "SmokeTest");
props.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.toString());
@@ -109,25 +113,29 @@ private static KafkaStreams createKafkaStreams(File stateDir, String kafka) {
props.put(ProducerConfig.ACKS_CONFIG, "all");
//TODO remove this config or set to smaller value when KIP-91 is merged
props.put(StreamsConfig.producerPrefix(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG), 80000);
+ return props;
+ }
- StreamsBuilder builder = new StreamsBuilder();
- Consumed stringIntConsumed = Consumed.with(stringSerde, intSerde);
- KStream source = builder.stream("data", stringIntConsumed);
- source.to(stringSerde, intSerde, "echo");
- KStream data = source.filter(new Predicate() {
+ private static KafkaStreams createKafkaStreams(final File stateDir, final String kafka) {
+ final StreamsBuilder builder = new StreamsBuilder();
+ final Consumed stringIntConsumed = Consumed.with(stringSerde, intSerde);
+ final KStream source = builder.stream("data", stringIntConsumed);
+ source.to("echo", Produced.with(stringSerde, intSerde));
+ final KStream data = source.filter(new Predicate() {
@Override
- public boolean test(String key, Integer value) {
+ public boolean test(final String key, final Integer value) {
return value == null || value != END;
}
});
data.process(SmokeTestUtil.printProcessorSupplier("data"));
// min
- KGroupedStream
- groupedData =
+ final KGroupedStream groupedData =
data.groupByKey(Serialized.with(stringSerde, intSerde));
- groupedData.aggregate(
+ groupedData
+ .windowedBy(TimeWindows.of(TimeUnit.DAYS.toMillis(1)))
+ .aggregate(
new Initializer() {
public Integer apply() {
return Integer.MAX_VALUE;
@@ -135,21 +143,24 @@ public Integer apply() {
},
new Aggregator() {
@Override
- public Integer apply(String aggKey, Integer value, Integer aggregate) {
+ public Integer apply(final String aggKey, final Integer value, final Integer aggregate) {
return (value < aggregate) ? value : aggregate;
}
},
- TimeWindows.of(TimeUnit.DAYS.toMillis(1)),
- intSerde, "uwin-min"
- ).toStream().map(
- new Unwindow()
- ).to(stringSerde, intSerde, "min");
+ Materialized.>as("uwin-min").withValueSerde(intSerde))
+ .toStream(new Unwindow())
+ .to("min", Produced.with(stringSerde, intSerde));
- KTable minTable = builder.table("min", stringIntConsumed);
+ final KTable minTable = builder.table(
+ "min",
+ Consumed.with(stringSerde, intSerde),
+ Materialized.>as("minStoreName"));
minTable.toStream().process(SmokeTestUtil.printProcessorSupplier("min"));
// max
- groupedData.aggregate(
+ groupedData
+ .windowedBy(TimeWindows.of(TimeUnit.DAYS.toMillis(2)))
+ .aggregate(
new Initializer() {
public Integer apply() {
return Integer.MIN_VALUE;
@@ -157,21 +168,24 @@ public Integer apply() {
},
new Aggregator() {
@Override
- public Integer apply(String aggKey, Integer value, Integer aggregate) {
+ public Integer apply(final String aggKey, final Integer value, final Integer aggregate) {
return (value > aggregate) ? value : aggregate;
}
},
- TimeWindows.of(TimeUnit.DAYS.toMillis(2)),
- intSerde, "uwin-max"
- ).toStream().map(
- new Unwindow()
- ).to(stringSerde, intSerde, "max");
+ Materialized.>as("uwin-max").withValueSerde(intSerde))
+ .toStream(new Unwindow())
+ .to("max", Produced.with(stringSerde, intSerde));
- KTable maxTable = builder.table("max", stringIntConsumed);
+ final KTable maxTable = builder.table(
+ "max",
+ Consumed.with(stringSerde, intSerde),
+ Materialized.>as("maxStoreName"));
maxTable.toStream().process(SmokeTestUtil.printProcessorSupplier("max"));
// sum
- groupedData.aggregate(
+ groupedData
+ .windowedBy(TimeWindows.of(TimeUnit.DAYS.toMillis(2)))
+ .aggregate(
new Initializer() {
public Long apply() {
return 0L;
@@ -179,70 +193,74 @@ public Long apply() {
},
new Aggregator() {
@Override
- public Long apply(String aggKey, Integer value, Long aggregate) {
+ public Long apply(final String aggKey, final Integer value, final Long aggregate) {
return (long) value + aggregate;
}
},
- TimeWindows.of(TimeUnit.DAYS.toMillis(2)),
- longSerde, "win-sum"
- ).toStream().map(
- new Unwindow()
- ).to(stringSerde, longSerde, "sum");
-
- Consumed stringLongConsumed = Consumed.with(stringSerde, longSerde);
- KTable sumTable = builder.table("sum", stringLongConsumed);
+ Materialized.>as("win-sum").withValueSerde(longSerde))
+ .toStream(new Unwindow())
+ .to("sum", Produced.with(stringSerde, longSerde));
+
+ final Consumed stringLongConsumed = Consumed.with(stringSerde, longSerde);
+ final KTable sumTable = builder.table("sum", stringLongConsumed);
sumTable.toStream().process(SmokeTestUtil.printProcessorSupplier("sum"));
+
// cnt
- groupedData.count(TimeWindows.of(TimeUnit.DAYS.toMillis(2)), "uwin-cnt")
- .toStream().map(
- new Unwindow()
- ).to(stringSerde, longSerde, "cnt");
+ groupedData
+ .windowedBy(TimeWindows.of(TimeUnit.DAYS.toMillis(2)))
+ .count(Materialized.>as("uwin-cnt"))
+ .toStream(new Unwindow())
+ .to("cnt", Produced.with(stringSerde, longSerde));
- KTable cntTable = builder.table("cnt", stringLongConsumed);
+ final KTable cntTable = builder.table(
+ "cnt",
+ Consumed.with(stringSerde, longSerde),
+ Materialized.>as("cntStoreName"));
cntTable.toStream().process(SmokeTestUtil.printProcessorSupplier("cnt"));
// dif
- maxTable.join(minTable,
+ maxTable
+ .join(
+ minTable,
new ValueJoiner() {
- public Integer apply(Integer value1, Integer value2) {
+ public Integer apply(final Integer value1, final Integer value2) {
return value1 - value2;
}
- }
- ).to(stringSerde, intSerde, "dif");
+ })
+ .toStream()
+ .to("dif", Produced.with(stringSerde, intSerde));
// avg
- sumTable.join(
+ sumTable
+ .join(
cntTable,
new ValueJoiner() {
- public Double apply(Long value1, Long value2) {
+ public Double apply(final Long value1, final Long value2) {
return (double) value1 / (double) value2;
}
- }
- ).to(stringSerde, doubleSerde, "avg");
+ })
+ .toStream()
+ .to("avg", Produced.with(stringSerde, doubleSerde));
// test repartition
- Agg agg = new Agg();
- cntTable.groupBy(agg.selector(),
- Serialized.with(stringSerde, longSerde)
- ).aggregate(agg.init(),
- agg.adder(),
- agg.remover(),
- Materialized.as(Stores.inMemoryKeyValueStore("cntByCnt"))
- .withKeySerde(Serdes.String())
- .withValueSerde(Serdes.Long())
- ).to(stringSerde, longSerde, "tagg");
-
- final KafkaStreams streamsClient = new KafkaStreams(builder.build(), props);
+ final Agg agg = new Agg();
+ cntTable.groupBy(agg.selector(), Serialized.with(stringSerde, longSerde))
+ .aggregate(agg.init(), agg.adder(), agg.remover(),
+ Materialized.as(Stores.inMemoryKeyValueStore("cntByCnt"))
+ .withKeySerde(Serdes.String())
+ .withValueSerde(Serdes.Long()))
+ .toStream()
+ .to("tagg", Produced.with(stringSerde, longSerde));
+
+ final KafkaStreams streamsClient = new KafkaStreams(builder.build(), getStreamsConfig(stateDir, kafka));
streamsClient.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
@Override
- public void uncaughtException(Thread t, Throwable e) {
+ public void uncaughtException(final Thread t, final Throwable e) {
System.out.println("FATAL: An unexpected exception is encountered on thread " + t + ": " + e);
-
streamsClient.close(30, TimeUnit.SECONDS);
}
});
return streamsClient;
}
-
}
diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java
index 882e9c0a58568..cba5fba6bd8aa 100644
--- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java
+++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java
@@ -130,53 +130,65 @@ public void run() {
System.out.println("shutdown");
}
- public static Map> generate(String kafka, final int numKeys, final int maxRecordsPerKey) {
+ public static Map> generate(final String kafka,
+ final int numKeys,
+ final int maxRecordsPerKey) {
+ return generate(kafka, numKeys, maxRecordsPerKey, true);
+ }
+
+ public static Map> generate(final String kafka,
+ final int numKeys,
+ final int maxRecordsPerKey,
+ final boolean autoTerminate) {
final Properties producerProps = new Properties();
producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, "SmokeTest");
producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka);
producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class);
producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class);
- // the next 4 config values make sure that all records are produced with no loss and
- // no duplicates
+ // the next 2 config values make sure that all records are produced with no loss and no duplicates
producerProps.put(ProducerConfig.RETRIES_CONFIG, Integer.MAX_VALUE);
producerProps.put(ProducerConfig.ACKS_CONFIG, "all");
producerProps.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, 80000);
- KafkaProducer producer = new KafkaProducer<>(producerProps);
+ final KafkaProducer producer = new KafkaProducer<>(producerProps);
int numRecordsProduced = 0;
- Map> allData = new HashMap<>();
- ValueList[] data = new ValueList[numKeys];
+ final Map> allData = new HashMap<>();
+ final ValueList[] data = new ValueList[numKeys];
for (int i = 0; i < numKeys; i++) {
data[i] = new ValueList(i, i + maxRecordsPerKey - 1);
allData.put(data[i].key, new HashSet());
}
- Random rand = new Random();
+ final Random rand = new Random();
- int remaining = data.length;
+ int remaining = 1; // dummy value must be positive if is false
+ if (autoTerminate) {
+ remaining = data.length;
+ }
List> needRetry = new ArrayList<>();
while (remaining > 0) {
- int index = rand.nextInt(remaining);
- String key = data[index].key;
+ final int index = autoTerminate ? rand.nextInt(remaining) : rand.nextInt(numKeys);
+ final String key = data[index].key;
int value = data[index].next();
- if (value < 0) {
+ if (autoTerminate && value < 0) {
remaining--;
data[index] = data[remaining];
} else {
- ProducerRecord record =
- new ProducerRecord<>("data", stringSerde.serializer().serialize("", key), intSerde.serializer().serialize("", value));
+ final ProducerRecord record =
+ new ProducerRecord<>("data", stringSerde.serializer().serialize("", key), intSerde.serializer().serialize("", value));
producer.send(record, new TestCallback(record, needRetry));
numRecordsProduced++;
allData.get(key).add(value);
- if (numRecordsProduced % 100 == 0)
+ if (numRecordsProduced % 100 == 0) {
System.out.println(numRecordsProduced + " records produced");
+ }
Utils.sleep(2);
}
}
diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java
index dc4c91b4097ed..87ca82918a951 100644
--- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java
+++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java
@@ -44,20 +44,15 @@ private static ProcessorSupplier