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.

+ +

Upgrading from 0.10.0.x to 1.1.0 requires an offline upgrade (rolling bounce upgrade is not supported)

+

Streams API changes in 1.1.0

diff --git a/docs/upgrade.html b/docs/upgrade.html index 3ac293d8498dd..26c4779a13304 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -63,6 +63,12 @@

Upgrading from 0.8.x, 0.9.x, 0.1 Hot-swaping the jar-file only might not work. +
Notable changes in 1.1.1
+
    +
  • New Kafka Streams configuration parameter upgrade.from added that allows rolling bounce upgrade from version 0.10.0.x
  • +
  • See the Kafka Streams upgrade guide for details about this new config. +
+

Notable changes in 1.1.0

Upgrading from 0.8.x or 0.9.x to 0.10.0.0

+

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 printProcessorSupplier(final St public Processor get() { return new AbstractProcessor() { private int numRecordsProcessed = 0; - private ProcessorContext context; @Override public void init(final ProcessorContext context) { System.out.println("initializing processor: topic=" + topic + " taskId=" + context.taskId()); numRecordsProcessed = 0; - this.context = context; } @Override public void process(final Object key, final Object value) { - if (printOffset) { - System.out.println(">>> " + context.offset()); - } numRecordsProcessed++; if (numRecordsProcessed % 100 == 0) { System.out.println(System.currentTimeMillis()); @@ -66,19 +61,19 @@ public void process(final Object key, final Object value) { } @Override - public void punctuate(final long timestamp) { } + public void punctuate(final long timestamp) {} @Override - public void close() { } + public void close() {} }; } }; } - public static final class Unwindow implements KeyValueMapper, V, KeyValue> { + public static final class Unwindow implements KeyValueMapper, V, K> { @Override - public KeyValue apply(final Windowed winKey, final V value) { - return new KeyValue<>(winKey.key(), value); + public K apply(final Windowed winKey, final V value) { + return winKey.key(); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java index 64597bdcda606..c0e345ff2b6d6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java @@ -23,7 +23,7 @@ public class StreamsSmokeTest { /** - * args ::= command kafka zookeeper stateDir + * args ::= command kafka zookeeper stateDir disableAutoTerminate * command := "run" | "process" * * @param args @@ -32,11 +32,13 @@ public static void main(String[] args) throws InterruptedException { String kafka = args[0]; String stateDir = args.length > 1 ? args[1] : null; String command = args.length > 2 ? args[2] : null; + boolean disableAutoTerminate = args.length > 3; - System.out.println("StreamsTest instance started"); + System.out.println("StreamsTest instance started (StreamsSmokeTest)"); System.out.println("command=" + command); System.out.println("kafka=" + kafka); System.out.println("stateDir=" + stateDir); + System.out.println("disableAutoTerminate=" + disableAutoTerminate); switch (command) { case "standalone": @@ -46,8 +48,12 @@ public static void main(String[] args) throws InterruptedException { // this starts the driver (data generation and result verification) final int numKeys = 10; final int maxRecordsPerKey = 500; - Map> allData = SmokeTestDriver.generate(kafka, numKeys, maxRecordsPerKey); - SmokeTestDriver.verify(kafka, allData, maxRecordsPerKey); + if (disableAutoTerminate) { + SmokeTestDriver.generate(kafka, numKeys, maxRecordsPerKey, false); + } else { + Map> allData = SmokeTestDriver.generate(kafka, numKeys, maxRecordsPerKey); + SmokeTestDriver.verify(kafka, allData, maxRecordsPerKey); + } break; case "process": // this starts a KafkaStreams client diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java new file mode 100644 index 0000000000000..5486374b62cb3 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.kstream.KStream; + +import java.util.Properties; + +public class StreamsUpgradeTest { + + @SuppressWarnings("unchecked") + public static void main(final String[] args) { + if (args.length < 2) { + System.err.println("StreamsUpgradeTest requires two argument (kafka-url, state-dir, [upgradeFrom: optional]) but only " + args.length + " provided: " + + (args.length > 0 ? args[0] : "")); + } + final String kafka = args[0]; + final String stateDir = args[1]; + final String upgradeFrom = args.length > 2 ? args[2] : null; + + System.out.println("StreamsTest instance started (StreamsUpgradeTest trunk)"); + System.out.println("kafka=" + kafka); + System.out.println("stateDir=" + stateDir); + System.out.println("upgradeFrom=" + upgradeFrom); + + final StreamsBuilder builder = new StreamsBuilder(); + final KStream dataStream = builder.stream("data"); + dataStream.process(SmokeTestUtil.printProcessorSupplier("data")); + dataStream.to("echo"); + + final Properties config = new Properties(); + config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsUpgradeTest"); + config.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + config.setProperty(StreamsConfig.STATE_DIR_CONFIG, stateDir); + config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000); + if (upgradeFrom != null) { + config.setProperty(StreamsConfig.UPGRADE_FROM_CONFIG, upgradeFrom); + } + + + final KafkaStreams streams = new KafkaStreams(builder.build(), config); + streams.start(); + + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + System.out.println("closing Kafka Streams instance"); + System.out.flush(); + streams.close(); + System.out.println("UPGRADE-TEST-CLIENT-CLOSED"); + System.out.flush(); + } + }); + } +} diff --git a/streams/upgrade-system-tests-0100/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java b/streams/upgrade-system-tests-0100/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java new file mode 100644 index 0000000000000..72d7f5a7b044a --- /dev/null +++ b/streams/upgrade-system-tests-0100/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +import java.util.Properties; + +public class StreamsUpgradeTest { + + @SuppressWarnings("unchecked") + public static void main(final String[] args) { + if (args.length < 3) { + System.err.println("StreamsUpgradeTest requires three argument (kafka-url, zookeeper-url, state-dir) but only " + args.length + " provided: " + + (args.length > 0 ? args[0] + " " : "") + + (args.length > 1 ? args[1] : "")); + } + final String kafka = args[0]; + final String zookeeper = args[1]; + final String stateDir = args[2]; + + System.out.println("StreamsTest instance started (StreamsUpgradeTest v0.10.0)"); + System.out.println("kafka=" + kafka); + System.out.println("zookeeper=" + zookeeper); + System.out.println("stateDir=" + stateDir); + + final KStreamBuilder builder = new KStreamBuilder(); + final KStream dataStream = builder.stream("data"); + dataStream.process(printProcessorSupplier()); + dataStream.to("echo"); + + final Properties config = new Properties(); + config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsUpgradeTest"); + config.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + config.setProperty(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, zookeeper); + config.setProperty(StreamsConfig.STATE_DIR_CONFIG, stateDir); + config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000); + + final KafkaStreams streams = new KafkaStreams(builder, config); + streams.start(); + + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + System.out.println("closing Kafka Streams instance"); + System.out.flush(); + streams.close(); + System.out.println("UPGRADE-TEST-CLIENT-CLOSED"); + System.out.flush(); + } + }); + } + + private static ProcessorSupplier printProcessorSupplier() { + return new ProcessorSupplier() { + public Processor get() { + return new AbstractProcessor() { + private int numRecordsProcessed = 0; + + @Override + public void init(final ProcessorContext context) { + System.out.println("initializing processor: topic=data taskId=" + context.taskId()); + numRecordsProcessed = 0; + } + + @Override + public void process(final K key, final V value) { + numRecordsProcessed++; + if (numRecordsProcessed % 100 == 0) { + System.out.println("processed " + numRecordsProcessed + " records from topic=data"); + } + } + + @Override + public void punctuate(final long timestamp) {} + + @Override + public void close() {} + }; + } + }; + } +} diff --git a/streams/upgrade-system-tests-0101/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java b/streams/upgrade-system-tests-0101/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java new file mode 100644 index 0000000000000..eebd0fab83ca0 --- /dev/null +++ b/streams/upgrade-system-tests-0101/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +import java.util.Properties; + +public class StreamsUpgradeTest { + + /** + * This test cannot be run executed, as long as Kafka 0.10.1.2 is not released + */ + @SuppressWarnings("unchecked") + public static void main(final String[] args) { + if (args.length < 3) { + System.err.println("StreamsUpgradeTest requires three argument (kafka-url, zookeeper-url, state-dir, [upgradeFrom: optional]) but only " + args.length + " provided: " + + (args.length > 0 ? args[0] + " " : "") + + (args.length > 1 ? args[1] : "")); + } + final String kafka = args[0]; + final String zookeeper = args[1]; + final String stateDir = args[2]; + final String upgradeFrom = args.length > 3 ? args[3] : null; + + System.out.println("StreamsTest instance started (StreamsUpgradeTest v0.10.1)"); + System.out.println("kafka=" + kafka); + System.out.println("zookeeper=" + zookeeper); + System.out.println("stateDir=" + stateDir); + System.out.println("upgradeFrom=" + upgradeFrom); + + final KStreamBuilder builder = new KStreamBuilder(); + final KStream dataStream = builder.stream("data"); + dataStream.process(printProcessorSupplier()); + dataStream.to("echo"); + + final Properties config = new Properties(); + config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsUpgradeTest"); + config.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + config.setProperty(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, zookeeper); + config.setProperty(StreamsConfig.STATE_DIR_CONFIG, stateDir); + config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000); + if (upgradeFrom != null) { + // TODO: because Kafka 0.10.1.2 is not released yet, thus `UPGRADE_FROM_CONFIG` is not available yet + //config.setProperty(StreamsConfig.UPGRADE_FROM_CONFIG, upgradeFrom); + config.setProperty("upgrade.from", upgradeFrom); + } + + final KafkaStreams streams = new KafkaStreams(builder, config); + streams.start(); + + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + System.out.println("closing Kafka Streams instance"); + System.out.flush(); + streams.close(); + System.out.println("UPGRADE-TEST-CLIENT-CLOSED"); + System.out.flush(); + } + }); + } + + private static ProcessorSupplier printProcessorSupplier() { + return new ProcessorSupplier() { + public Processor get() { + return new AbstractProcessor() { + private int numRecordsProcessed = 0; + + @Override + public void init(final ProcessorContext context) { + System.out.println("initializing processor: topic=data taskId=" + context.taskId()); + numRecordsProcessed = 0; + } + + @Override + public void process(final K key, final V value) { + numRecordsProcessed++; + if (numRecordsProcessed % 100 == 0) { + System.out.println("processed " + numRecordsProcessed + " records from topic=data"); + } + } + + @Override + public void punctuate(final long timestamp) {} + + @Override + public void close() {} + }; + } + }; + } +} diff --git a/streams/upgrade-system-tests-0102/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java b/streams/upgrade-system-tests-0102/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java new file mode 100644 index 0000000000000..18240f04ff1c5 --- /dev/null +++ b/streams/upgrade-system-tests-0102/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +import java.util.Properties; + +public class StreamsUpgradeTest { + + /** + * This test cannot be run executed, as long as Kafka 0.10.2.2 is not released + */ + @SuppressWarnings("unchecked") + public static void main(final String[] args) { + if (args.length < 2) { + System.err.println("StreamsUpgradeTest requires three argument (kafka-url, state-dir, [upgradeFrom: optional]) but only " + args.length + " provided: " + + (args.length > 0 ? args[0] : "")); + } + final String kafka = args[0]; + final String stateDir = args[1]; + final String upgradeFrom = args.length > 2 ? args[2] : null; + + System.out.println("StreamsTest instance started (StreamsUpgradeTest v0.10.2)"); + System.out.println("kafka=" + kafka); + System.out.println("stateDir=" + stateDir); + System.out.println("upgradeFrom=" + upgradeFrom); + + final KStreamBuilder builder = new KStreamBuilder(); + final KStream dataStream = builder.stream("data"); + dataStream.process(printProcessorSupplier()); + dataStream.to("echo"); + + final Properties config = new Properties(); + config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsUpgradeTest"); + config.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + config.setProperty(StreamsConfig.STATE_DIR_CONFIG, stateDir); + config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000); + if (upgradeFrom != null) { + // TODO: because Kafka 0.10.2.2 is not released yet, thus `UPGRADE_FROM_CONFIG` is not available yet + //config.setProperty(StreamsConfig.UPGRADE_FROM_CONFIG, upgradeFrom); + config.setProperty("upgrade.from", upgradeFrom); + } + + final KafkaStreams streams = new KafkaStreams(builder, config); + streams.start(); + + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + streams.close(); + System.out.println("UPGRADE-TEST-CLIENT-CLOSED"); + System.out.flush(); + } + }); + } + + private static ProcessorSupplier printProcessorSupplier() { + return new ProcessorSupplier() { + public Processor get() { + return new AbstractProcessor() { + private int numRecordsProcessed = 0; + + @Override + public void init(final ProcessorContext context) { + System.out.println("initializing processor: topic=data taskId=" + context.taskId()); + numRecordsProcessed = 0; + } + + @Override + public void process(final K key, final V value) { + numRecordsProcessed++; + if (numRecordsProcessed % 100 == 0) { + System.out.println("processed " + numRecordsProcessed + " records from topic=data"); + } + } + + @Override + public void punctuate(final long timestamp) {} + + @Override + public void close() {} + }; + } + }; + } +} diff --git a/streams/upgrade-system-tests-0110/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java b/streams/upgrade-system-tests-0110/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java new file mode 100644 index 0000000000000..779021d867245 --- /dev/null +++ b/streams/upgrade-system-tests-0110/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +import java.util.Properties; + +public class StreamsUpgradeTest { + + /** + * This test cannot be run executed, as long as Kafka 0.11.0.3 is not released + */ + @SuppressWarnings("unchecked") + public static void main(final String[] args) { + if (args.length < 2) { + System.err.println("StreamsUpgradeTest requires three argument (kafka-url, state-dir, [upgradeFrom: optional]) but only " + args.length + " provided: " + + (args.length > 0 ? args[0] : "")); + } + final String kafka = args[0]; + final String stateDir = args[1]; + final String upgradeFrom = args.length > 2 ? args[2] : null; + + System.out.println("StreamsTest instance started (StreamsUpgradeTest v0.11.0)"); + System.out.println("kafka=" + kafka); + System.out.println("stateDir=" + stateDir); + System.out.println("upgradeFrom=" + upgradeFrom); + + final KStreamBuilder builder = new KStreamBuilder(); + final KStream dataStream = builder.stream("data"); + dataStream.process(printProcessorSupplier()); + dataStream.to("echo"); + + final Properties config = new Properties(); + config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsUpgradeTest"); + config.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + config.setProperty(StreamsConfig.STATE_DIR_CONFIG, stateDir); + config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000); + if (upgradeFrom != null) { + // TODO: because Kafka 0.11.0.3 is not released yet, thus `UPGRADE_FROM_CONFIG` is not available yet + //config.setProperty(StreamsConfig.UPGRADE_FROM_CONFIG, upgradeFrom); + config.setProperty("upgrade.from", upgradeFrom); + } + + final KafkaStreams streams = new KafkaStreams(builder, config); + streams.start(); + + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + streams.close(); + System.out.println("UPGRADE-TEST-CLIENT-CLOSED"); + System.out.flush(); + } + }); + } + + private static ProcessorSupplier printProcessorSupplier() { + return new ProcessorSupplier() { + public Processor get() { + return new AbstractProcessor() { + private int numRecordsProcessed = 0; + + @Override + public void init(final ProcessorContext context) { + System.out.println("initializing processor: topic=data taskId=" + context.taskId()); + numRecordsProcessed = 0; + } + + @Override + public void process(final K key, final V value) { + numRecordsProcessed++; + if (numRecordsProcessed % 100 == 0) { + System.out.println("processed " + numRecordsProcessed + " records from topic=data"); + } + } + + @Override + public void punctuate(final long timestamp) {} + + @Override + public void close() {} + }; + } + }; + } +} diff --git a/streams/upgrade-system-tests-10/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java b/streams/upgrade-system-tests-10/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java new file mode 100644 index 0000000000000..4d008c2513655 --- /dev/null +++ b/streams/upgrade-system-tests-10/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +import java.util.Properties; + +public class StreamsUpgradeTest { + + /** + * This test cannot be run executed, as long as Kafka 1.0.2 is not released + */ + @SuppressWarnings("unchecked") + public static void main(final String[] args) { + if (args.length < 2) { + System.err.println("StreamsUpgradeTest requires three argument (kafka-url, state-dir, [upgradeFrom: optional]) but only " + args.length + " provided: " + + (args.length > 0 ? args[0] : "")); + } + final String kafka = args[0]; + final String stateDir = args[1]; + final String upgradeFrom = args.length > 2 ? args[2] : null; + + System.out.println("StreamsTest instance started (StreamsUpgradeTest v1.0)"); + System.out.println("kafka=" + kafka); + System.out.println("stateDir=" + stateDir); + System.out.println("upgradeFrom=" + upgradeFrom); + + final StreamsBuilder builder = new StreamsBuilder(); + final KStream dataStream = builder.stream("data"); + dataStream.process(printProcessorSupplier()); + dataStream.to("echo"); + + final Properties config = new Properties(); + config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsUpgradeTest"); + config.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + config.setProperty(StreamsConfig.STATE_DIR_CONFIG, stateDir); + config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000); + if (upgradeFrom != null) { + // TODO: because Kafka 1.0.2 is not released yet, thus `UPGRADE_FROM_CONFIG` is not available yet + //config.setProperty(StreamsConfig.UPGRADE_FROM_CONFIG, upgradeFrom); + config.setProperty("upgrade.from", upgradeFrom); + } + + final KafkaStreams streams = new KafkaStreams(builder.build(), config); + streams.start(); + + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + streams.close(); + System.out.println("UPGRADE-TEST-CLIENT-CLOSED"); + System.out.flush(); + } + }); + } + + private static ProcessorSupplier printProcessorSupplier() { + return new ProcessorSupplier() { + public Processor get() { + return new AbstractProcessor() { + private int numRecordsProcessed = 0; + + @Override + public void init(final ProcessorContext context) { + System.out.println("initializing processor: topic=data taskId=" + context.taskId()); + numRecordsProcessed = 0; + } + + @Override + public void process(final K key, final V value) { + numRecordsProcessed++; + if (numRecordsProcessed % 100 == 0) { + System.out.println("processed " + numRecordsProcessed + " records from topic=data"); + } + } + + @Override + public void punctuate(final long timestamp) {} + + @Override + public void close() {} + }; + } + }; + } +} diff --git a/tests/docker/Dockerfile b/tests/docker/Dockerfile index 149e3911bc6f5..b6c499756a8a0 100644 --- a/tests/docker/Dockerfile +++ b/tests/docker/Dockerfile @@ -39,13 +39,15 @@ COPY ./ssh-config /root/.ssh/config RUN ssh-keygen -q -t rsa -N '' -f /root/.ssh/id_rsa && cp -f /root/.ssh/id_rsa.pub /root/.ssh/authorized_keys # Install binary test dependencies. +# we use the same versions as in vagrant/base.sh ARG KAFKA_MIRROR="https://s3-us-west-2.amazonaws.com/kafka-packages" -RUN mkdir -p "/opt/kafka-0.8.2.2" && curl -s "$KAFKA_MIRROR/kafka_2.10-0.8.2.2.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.8.2.2" +RUN mkdir -p "/opt/kafka-0.8.2.2" && curl -s "$KAFKA_MIRROR/kafka_2.11-0.8.2.2.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.8.2.2" RUN mkdir -p "/opt/kafka-0.9.0.1" && curl -s "$KAFKA_MIRROR/kafka_2.11-0.9.0.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.9.0.1" RUN mkdir -p "/opt/kafka-0.10.0.1" && curl -s "$KAFKA_MIRROR/kafka_2.11-0.10.0.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.10.0.1" RUN mkdir -p "/opt/kafka-0.10.1.1" && curl -s "$KAFKA_MIRROR/kafka_2.11-0.10.1.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.10.1.1" RUN mkdir -p "/opt/kafka-0.10.2.1" && curl -s "$KAFKA_MIRROR/kafka_2.11-0.10.2.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.10.2.1" -RUN mkdir -p "/opt/kafka-0.11.0.0" && curl -s "$KAFKA_MIRROR/kafka_2.11-0.11.0.0.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.11.0.0" +RUN mkdir -p "/opt/kafka-0.11.0.2" && curl -s "$KAFKA_MIRROR/kafka_2.11-0.11.0.2.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.11.0.2" +RUN mkdir -p "/opt/kafka-1.0.1" && curl -s "$KAFKA_MIRROR/kafka_2.11-1.0.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-1.0.1" # The version of Kibosh to use for testing. # If you update this, also update vagrant/base.sy diff --git a/tests/kafkatest/services/streams.py b/tests/kafkatest/services/streams.py index 54e26bfbc2be0..0ed3a4290fb9d 100644 --- a/tests/kafkatest/services/streams.py +++ b/tests/kafkatest/services/streams.py @@ -20,6 +20,7 @@ from ducktape.utils.util import wait_until from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin +from kafkatest.version import LATEST_0_10_0, LATEST_0_10_1 class StreamsTestBaseService(KafkaPathResolverMixin, Service): @@ -33,6 +34,8 @@ class StreamsTestBaseService(KafkaPathResolverMixin, Service): LOG4J_CONFIG_FILE = os.path.join(PERSISTENT_ROOT, "tools-log4j.properties") PID_FILE = os.path.join(PERSISTENT_ROOT, "streams.pid") + CLEAN_NODE_ENABLED = True + logs = { "streams_log": { "path": LOG_FILE, @@ -43,6 +46,114 @@ class StreamsTestBaseService(KafkaPathResolverMixin, Service): "streams_stderr": { "path": STDERR_FILE, "collect_default": True}, + "streams_log.0-1": { + "path": LOG_FILE + ".0-1", + "collect_default": True}, + "streams_stdout.0-1": { + "path": STDOUT_FILE + ".0-1", + "collect_default": True}, + "streams_stderr.0-1": { + "path": STDERR_FILE + ".0-1", + "collect_default": True}, + "streams_log.0-2": { + "path": LOG_FILE + ".0-2", + "collect_default": True}, + "streams_stdout.0-2": { + "path": STDOUT_FILE + ".0-2", + "collect_default": True}, + "streams_stderr.0-2": { + "path": STDERR_FILE + ".0-2", + "collect_default": True}, + "streams_log.0-3": { + "path": LOG_FILE + ".0-3", + "collect_default": True}, + "streams_stdout.0-3": { + "path": STDOUT_FILE + ".0-3", + "collect_default": True}, + "streams_stderr.0-3": { + "path": STDERR_FILE + ".0-3", + "collect_default": True}, + "streams_log.0-4": { + "path": LOG_FILE + ".0-4", + "collect_default": True}, + "streams_stdout.0-4": { + "path": STDOUT_FILE + ".0-4", + "collect_default": True}, + "streams_stderr.0-4": { + "path": STDERR_FILE + ".0-4", + "collect_default": True}, + "streams_log.0-5": { + "path": LOG_FILE + ".0-5", + "collect_default": True}, + "streams_stdout.0-5": { + "path": STDOUT_FILE + ".0-5", + "collect_default": True}, + "streams_stderr.0-5": { + "path": STDERR_FILE + ".0-5", + "collect_default": True}, + "streams_log.0-6": { + "path": LOG_FILE + ".0-6", + "collect_default": True}, + "streams_stdout.0-6": { + "path": STDOUT_FILE + ".0-6", + "collect_default": True}, + "streams_stderr.0-6": { + "path": STDERR_FILE + ".0-6", + "collect_default": True}, + "streams_log.1-1": { + "path": LOG_FILE + ".1-1", + "collect_default": True}, + "streams_stdout.1-1": { + "path": STDOUT_FILE + ".1-1", + "collect_default": True}, + "streams_stderr.1-1": { + "path": STDERR_FILE + ".1-1", + "collect_default": True}, + "streams_log.1-2": { + "path": LOG_FILE + ".1-2", + "collect_default": True}, + "streams_stdout.1-2": { + "path": STDOUT_FILE + ".1-2", + "collect_default": True}, + "streams_stderr.1-2": { + "path": STDERR_FILE + ".1-2", + "collect_default": True}, + "streams_log.1-3": { + "path": LOG_FILE + ".1-3", + "collect_default": True}, + "streams_stdout.1-3": { + "path": STDOUT_FILE + ".1-3", + "collect_default": True}, + "streams_stderr.1-3": { + "path": STDERR_FILE + ".1-3", + "collect_default": True}, + "streams_log.1-4": { + "path": LOG_FILE + ".1-4", + "collect_default": True}, + "streams_stdout.1-4": { + "path": STDOUT_FILE + ".1-4", + "collect_default": True}, + "streams_stderr.1-4": { + "path": STDERR_FILE + ".1-4", + "collect_default": True}, + "streams_log.1-5": { + "path": LOG_FILE + ".1-5", + "collect_default": True}, + "streams_stdout.1-5": { + "path": STDOUT_FILE + ".1-5", + "collect_default": True}, + "streams_stderr.1-5": { + "path": STDERR_FILE + ".1-5", + "collect_default": True}, + "streams_log.1-6": { + "path": LOG_FILE + ".1-6", + "collect_default": True}, + "streams_stdout.1-6": { + "path": STDOUT_FILE + ".1-6", + "collect_default": True}, + "streams_stderr.1-6": { + "path": STDERR_FILE + ".1-6", + "collect_default": True}, } def __init__(self, test_context, kafka, streams_class_name, user_test_args, user_test_args1=None, user_test_args2=None, user_test_args3=None): @@ -108,7 +219,8 @@ def wait_node(self, node, timeout_sec=None): def clean_node(self, node): node.account.kill_process("streams", clean_shutdown=False, allow_fail=True) - node.account.ssh("rm -rf " + self.PERSISTENT_ROOT, allow_fail=False) + if self.CLEAN_NODE_ENABLED: + node.account.ssh("rm -rf " + self.PERSISTENT_ROOT, allow_fail=False) def start_cmd(self, node): args = self.args.copy() @@ -170,7 +282,28 @@ def clean_node(self, node): class StreamsSmokeTestDriverService(StreamsSmokeTestBaseService): def __init__(self, test_context, kafka): super(StreamsSmokeTestDriverService, self).__init__(test_context, kafka, "run") + self.DISABLE_AUTO_TERMINATE = "" + + def disable_auto_terminate(self): + self.DISABLE_AUTO_TERMINATE = "disableAutoTerminate" + + def start_cmd(self, node): + args = self.args.copy() + args['kafka'] = self.kafka.bootstrap_servers() + args['state_dir'] = self.PERSISTENT_ROOT + args['stdout'] = self.STDOUT_FILE + args['stderr'] = self.STDERR_FILE + args['pidfile'] = self.PID_FILE + args['log4j'] = self.LOG4J_CONFIG_FILE + args['disable_auto_terminate'] = self.DISABLE_AUTO_TERMINATE + args['kafka_run_class'] = self.path.script("kafka-run-class.sh", node) + cmd = "( export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%(log4j)s\"; " \ + "INCLUDE_TEST_JARS=true %(kafka_run_class)s %(streams_class_name)s " \ + " %(kafka)s %(state_dir)s %(user_test_args)s %(disable_auto_terminate)s" \ + " & echo $! >&3 ) 1>> %(stdout)s 2>> %(stderr)s 3> %(pidfile)s" % args + + return cmd class StreamsSmokeTestJobRunnerService(StreamsSmokeTestBaseService): def __init__(self, test_context, kafka): @@ -218,3 +351,40 @@ def __init__(self, test_context, kafka, configs): kafka, "org.apache.kafka.streams.tests.StreamsBrokerDownResilienceTest", configs) +class StreamsUpgradeTestJobRunnerService(StreamsTestBaseService): + def __init__(self, test_context, kafka): + super(StreamsUpgradeTestJobRunnerService, self).__init__(test_context, + kafka, + "org.apache.kafka.streams.tests.StreamsUpgradeTest", + "") + self.UPGRADE_FROM = "" + + def set_version(self, kafka_streams_version): + self.KAFKA_STREAMS_VERSION = kafka_streams_version + + def set_upgrade_from(self, upgrade_from): + self.UPGRADE_FROM = upgrade_from + + def start_cmd(self, node): + args = self.args.copy() + args['kafka'] = self.kafka.bootstrap_servers() + if self.KAFKA_STREAMS_VERSION == str(LATEST_0_10_0) or self.KAFKA_STREAMS_VERSION == str(LATEST_0_10_1): + args['zk'] = self.kafka.zk.connect_setting() + else: + args['zk'] = "" + args['state_dir'] = self.PERSISTENT_ROOT + args['stdout'] = self.STDOUT_FILE + args['stderr'] = self.STDERR_FILE + args['pidfile'] = self.PID_FILE + args['log4j'] = self.LOG4J_CONFIG_FILE + args['version'] = self.KAFKA_STREAMS_VERSION + args['upgrade_from'] = self.UPGRADE_FROM + args['kafka_run_class'] = self.path.script("kafka-run-class.sh", node) + + cmd = "( export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%(log4j)s\"; " \ + "INCLUDE_TEST_JARS=true UPGRADE_KAFKA_STREAMS_TEST_VERSION=%(version)s " \ + " %(kafka_run_class)s %(streams_class_name)s " \ + " %(kafka)s %(zk)s %(state_dir)s %(user_test_args)s %(upgrade_from)s" \ + " & echo $! >&3 ) 1>> %(stdout)s 2>> %(stderr)s 3> %(pidfile)s" % args + + return cmd diff --git a/tests/kafkatest/tests/streams/streams_upgrade_test.py b/tests/kafkatest/tests/streams/streams_upgrade_test.py index 81b7ffe7047c9..77833a92be49e 100644 --- a/tests/kafkatest/tests/streams/streams_upgrade_test.py +++ b/tests/kafkatest/tests/streams/streams_upgrade_test.py @@ -15,21 +15,48 @@ from ducktape.mark.resource import cluster from ducktape.tests.test import Test -from ducktape.mark import parametrize, ignore +from ducktape.mark import ignore, matrix, parametrize from kafkatest.services.kafka import KafkaService from kafkatest.services.zookeeper import ZookeeperService -from kafkatest.services.streams import StreamsSmokeTestDriverService, StreamsSmokeTestJobRunnerService -from kafkatest.version import LATEST_0_10_1, LATEST_0_10_2, LATEST_0_11_0, DEV_BRANCH, KafkaVersion +from kafkatest.services.streams import StreamsSmokeTestDriverService, StreamsSmokeTestJobRunnerService, StreamsUpgradeTestJobRunnerService +from kafkatest.version import LATEST_0_10_0, LATEST_0_10_1, LATEST_0_10_2, LATEST_0_11_0, LATEST_1_0, DEV_BRANCH, DEV_VERSION, KafkaVersion +import random import time +broker_upgrade_versions = [str(LATEST_0_10_1), str(LATEST_0_10_2), str(LATEST_0_11_0), str(LATEST_1_0), str(DEV_BRANCH)] +simple_upgrade_versions_metadata_version_2 = [str(LATEST_0_10_1), str(LATEST_0_10_2), str(LATEST_0_11_0), str(LATEST_1_0), str(DEV_VERSION)] class StreamsUpgradeTest(Test): """ - Tests rolling upgrades and downgrades of the Kafka Streams library. + Test upgrading Kafka Streams (all version combination) + If metadata was changes, upgrade is more difficult + Metadata version was bumped in 0.10.1.0 """ def __init__(self, test_context): super(StreamsUpgradeTest, self).__init__(test_context) + self.topics = { + 'echo' : { 'partitions': 5 }, + 'data' : { 'partitions': 5 }, + } + + def perform_broker_upgrade(self, to_version): + self.logger.info("First pass bounce - rolling broker upgrade") + for node in self.kafka.nodes: + self.kafka.stop_node(node) + node.version = KafkaVersion(to_version) + self.kafka.start_node(node) + + @cluster(num_nodes=6) + @matrix(from_version=broker_upgrade_versions, to_version=broker_upgrade_versions) + def test_upgrade_downgrade_brokers(self, from_version, to_version): + """ + Start a smoke test client then perform rolling upgrades on the broker. + """ + + if from_version == to_version: + return + self.replication = 3 self.partitions = 1 self.isr = 2 @@ -55,45 +82,7 @@ def __init__(self, test_context): 'tagg' : { 'partitions': self.partitions, 'replication-factor': self.replication, 'configs': {"min.insync.replicas": self.isr} } } - - - def perform_streams_upgrade(self, to_version): - self.logger.info("First pass bounce - rolling streams upgrade") - - # get the node running the streams app - node = self.processor1.node - self.processor1.stop() - - # change it's version. This will automatically make it pick up a different - # JAR when it starts again - node.version = KafkaVersion(to_version) - self.processor1.start() - - def perform_broker_upgrade(self, to_version): - self.logger.info("First pass bounce - rolling broker upgrade") - for node in self.kafka.nodes: - self.kafka.stop_node(node) - node.version = KafkaVersion(to_version) - self.kafka.start_node(node) - - @cluster(num_nodes=6) - @parametrize(from_version=str(LATEST_0_10_1), to_version=str(DEV_BRANCH)) - @parametrize(from_version=str(LATEST_0_10_2), to_version=str(DEV_BRANCH)) - @parametrize(from_version=str(LATEST_0_10_1), to_version=str(LATEST_0_11_0)) - @parametrize(from_version=str(LATEST_0_10_2), to_version=str(LATEST_0_11_0)) - @parametrize(from_version=str(LATEST_0_11_0), to_version=str(LATEST_0_10_2)) - @parametrize(from_version=str(DEV_BRANCH), to_version=str(LATEST_0_10_2)) - def test_upgrade_downgrade_streams(self, from_version, to_version): - """ - Start a smoke test client, then abort (kill -9) and restart it a few times. - Ensure that all records are delivered. - - Note, that just like tests/core/upgrade_test.py, a prerequisite for this test to succeed - if the inclusion of all parametrized versions of kafka in kafka/vagrant/base.sh - (search for get_kafka()). For streams in particular, that means that someone has manually - copies the kafka-stream-$version-test.jar in the right S3 bucket as shown in base.sh. - """ # Setup phase self.zk = ZookeeperService(self.test_context, num_nodes=1) self.zk.start() @@ -108,13 +97,12 @@ def test_upgrade_downgrade_streams(self, from_version, to_version): self.driver = StreamsSmokeTestDriverService(self.test_context, self.kafka) self.processor1 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka) - self.driver.start() self.processor1.start() time.sleep(15) - self.perform_streams_upgrade(to_version) + self.perform_broker_upgrade(to_version) time.sleep(15) self.driver.wait() @@ -126,42 +114,241 @@ def test_upgrade_downgrade_streams(self, from_version, to_version): node.account.ssh("grep ALL-RECORDS-DELIVERED %s" % self.driver.STDOUT_FILE, allow_fail=False) self.processor1.node.account.ssh_capture("grep SMOKE-TEST-CLIENT-CLOSED %s" % self.processor1.STDOUT_FILE, allow_fail=False) + @matrix(from_version=simple_upgrade_versions_metadata_version_2, to_version=simple_upgrade_versions_metadata_version_2) + def test_simple_upgrade_downgrade(self, from_version, to_version): + """ + Starts 3 KafkaStreams instances with , and upgrades one-by-one to + """ + if from_version == to_version: + return - @cluster(num_nodes=6) - @parametrize(from_version=str(LATEST_0_10_2), to_version=str(DEV_BRANCH)) - def test_upgrade_brokers(self, from_version, to_version): + self.zk = ZookeeperService(self.test_context, num_nodes=1) + self.zk.start() + + self.kafka = KafkaService(self.test_context, num_nodes=1, zk=self.zk, topics=self.topics) + self.kafka.start() + + self.driver = StreamsSmokeTestDriverService(self.test_context, self.kafka) + self.driver.disable_auto_terminate() + self.processor1 = StreamsUpgradeTestJobRunnerService(self.test_context, self.kafka) + self.processor2 = StreamsUpgradeTestJobRunnerService(self.test_context, self.kafka) + self.processor3 = StreamsUpgradeTestJobRunnerService(self.test_context, self.kafka) + + self.driver.start() + self.start_all_nodes_with(from_version) + + self.processors = [self.processor1, self.processor2, self.processor3] + + counter = 1 + random.seed() + + # upgrade one-by-one via rolling bounce + random.shuffle(self.processors) + for p in self.processors: + p.CLEAN_NODE_ENABLED = False + self.do_rolling_bounce(p, "", to_version, counter) + counter = counter + 1 + + # shutdown + self.driver.stop() + self.driver.wait() + + random.shuffle(self.processors) + for p in self.processors: + node = p.node + with node.account.monitor_log(p.STDOUT_FILE) as monitor: + p.stop() + monitor.wait_until("UPGRADE-TEST-CLIENT-CLOSED", + timeout_sec=60, + err_msg="Never saw output 'UPGRADE-TEST-CLIENT-CLOSED' on" + str(node.account)) + + self.driver.stop() + + #@parametrize(new_version=str(LATEST_0_10_1)) we cannot run this test until Kafka 0.10.1.2 is released + #@parametrize(new_version=str(LATEST_0_10_2)) we cannot run this test until Kafka 0.10.2.2 is released + #@parametrize(new_version=str(LATEST_0_11_0)) we cannot run this test until Kafka 0.11.0.3 is released + #@parametrize(new_version=str(LATEST_1_0)) we cannot run this test until Kafka 1.0.2 is released + #@parametrize(new_version=str(LATEST_1_1)) we cannot run this test until Kafka 1.1.1 is released + @parametrize(new_version=str(DEV_VERSION)) + def test_metadata_upgrade(self, new_version): """ - Start a smoke test client then perform rolling upgrades on the broker. + Starts 3 KafkaStreams instances with version 0.10.0, and upgrades one-by-one to """ - # Setup phase + self.zk = ZookeeperService(self.test_context, num_nodes=1) self.zk.start() - # number of nodes needs to be >= 3 for the smoke test - self.kafka = KafkaService(self.test_context, num_nodes=3, - zk=self.zk, version=KafkaVersion(from_version), topics=self.topics) + self.kafka = KafkaService(self.test_context, num_nodes=1, zk=self.zk, topics=self.topics) self.kafka.start() - - # allow some time for topics to be created - time.sleep(10) - + self.driver = StreamsSmokeTestDriverService(self.test_context, self.kafka) - self.processor1 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka) + self.driver.disable_auto_terminate() + self.processor1 = StreamsUpgradeTestJobRunnerService(self.test_context, self.kafka) + self.processor2 = StreamsUpgradeTestJobRunnerService(self.test_context, self.kafka) + self.processor3 = StreamsUpgradeTestJobRunnerService(self.test_context, self.kafka) - self.driver.start() - self.processor1.start() - time.sleep(15) + self.start_all_nodes_with(str(LATEST_0_10_0)) - self.perform_broker_upgrade(to_version) + self.processors = [self.processor1, self.processor2, self.processor3] - time.sleep(15) + counter = 1 + random.seed() + + # first rolling bounce + random.shuffle(self.processors) + for p in self.processors: + p.CLEAN_NODE_ENABLED = False + self.do_rolling_bounce(p, "0.10.0", new_version, counter) + counter = counter + 1 + + # second rolling bounce + random.shuffle(self.processors) + for p in self.processors: + self.do_rolling_bounce(p, "", new_version, counter) + counter = counter + 1 + + # shutdown + self.driver.stop() self.driver.wait() + + random.shuffle(self.processors) + for p in self.processors: + node = p.node + with node.account.monitor_log(p.STDOUT_FILE) as monitor: + p.stop() + monitor.wait_until("UPGRADE-TEST-CLIENT-CLOSED", + timeout_sec=60, + err_msg="Never saw output 'UPGRADE-TEST-CLIENT-CLOSED' on" + str(node.account)) + self.driver.stop() - self.processor1.stop() + def start_all_nodes_with(self, version): + # start first with + self.prepare_for(self.processor1, version) + node1 = self.processor1.node + with node1.account.monitor_log(self.processor1.STDOUT_FILE) as monitor: + with node1.account.monitor_log(self.processor1.LOG_FILE) as log_monitor: + self.processor1.start() + log_monitor.wait_until("Kafka version : " + version, + timeout_sec=60, + err_msg="Could not detect Kafka Streams version " + version + " " + str(node1.account)) + monitor.wait_until("processed 100 records from topic", + timeout_sec=60, + err_msg="Never saw output 'processed 100 records from topic' on" + str(node1.account)) - node = self.driver.node - node.account.ssh("grep ALL-RECORDS-DELIVERED %s" % self.driver.STDOUT_FILE, allow_fail=False) - self.processor1.node.account.ssh_capture("grep SMOKE-TEST-CLIENT-CLOSED %s" % self.processor1.STDOUT_FILE, allow_fail=False) + # start second with + self.prepare_for(self.processor2, version) + node2 = self.processor2.node + with node1.account.monitor_log(self.processor1.STDOUT_FILE) as first_monitor: + with node2.account.monitor_log(self.processor2.STDOUT_FILE) as second_monitor: + with node2.account.monitor_log(self.processor2.LOG_FILE) as log_monitor: + self.processor2.start() + log_monitor.wait_until("Kafka version : " + version, + timeout_sec=60, + err_msg="Could not detect Kafka Streams version " + version + " " + str(node2.account)) + first_monitor.wait_until("processed 100 records from topic", + timeout_sec=60, + err_msg="Never saw output 'processed 100 records from topic' on" + str(node1.account)) + second_monitor.wait_until("processed 100 records from topic", + timeout_sec=60, + err_msg="Never saw output 'processed 100 records from topic' on" + str(node2.account)) + + # start third with + self.prepare_for(self.processor3, version) + node3 = self.processor3.node + with node1.account.monitor_log(self.processor1.STDOUT_FILE) as first_monitor: + with node2.account.monitor_log(self.processor2.STDOUT_FILE) as second_monitor: + with node3.account.monitor_log(self.processor3.STDOUT_FILE) as third_monitor: + with node3.account.monitor_log(self.processor3.LOG_FILE) as log_monitor: + self.processor3.start() + log_monitor.wait_until("Kafka version : " + version, + timeout_sec=60, + err_msg="Could not detect Kafka Streams version " + version + " " + str(node3.account)) + first_monitor.wait_until("processed 100 records from topic", + timeout_sec=60, + err_msg="Never saw output 'processed 100 records from topic' on" + str(node1.account)) + second_monitor.wait_until("processed 100 records from topic", + timeout_sec=60, + err_msg="Never saw output 'processed 100 records from topic' on" + str(node2.account)) + third_monitor.wait_until("processed 100 records from topic", + timeout_sec=60, + err_msg="Never saw output 'processed 100 records from topic' on" + str(node3.account)) + + @staticmethod + def prepare_for(processor, version): + processor.node.account.ssh("rm -rf " + processor.PERSISTENT_ROOT, allow_fail=False) + if version == str(DEV_VERSION): + processor.set_version("") # set to TRUNK + else: + processor.set_version(version) + + def do_rolling_bounce(self, processor, upgrade_from, new_version, counter): + first_other_processor = None + second_other_processor = None + for p in self.processors: + if p != processor: + if first_other_processor is None: + first_other_processor = p + else: + second_other_processor = p + + node = processor.node + first_other_node = first_other_processor.node + second_other_node = second_other_processor.node + + # stop processor and wait for rebalance of others + with first_other_node.account.monitor_log(first_other_processor.STDOUT_FILE) as first_other_monitor: + with second_other_node.account.monitor_log(second_other_processor.STDOUT_FILE) as second_other_monitor: + processor.stop() + first_other_monitor.wait_until("processed 100 records from topic", + timeout_sec=60, + err_msg="Never saw output 'processed 100 records from topic' on" + str(first_other_node.account)) + second_other_monitor.wait_until("processed 100 records from topic", + timeout_sec=60, + err_msg="Never saw output 'processed 100 records from topic' on" + str(second_other_node.account)) + node.account.ssh_capture("grep UPGRADE-TEST-CLIENT-CLOSED %s" % processor.STDOUT_FILE, allow_fail=False) + + if upgrade_from == "": # upgrade disabled -- second round of rolling bounces + roll_counter = ".1-" # second round of rolling bounces + else: + roll_counter = ".0-" # first round of rolling boundes + + node.account.ssh("mv " + processor.STDOUT_FILE + " " + processor.STDOUT_FILE + roll_counter + str(counter), allow_fail=False) + node.account.ssh("mv " + processor.STDERR_FILE + " " + processor.STDERR_FILE + roll_counter + str(counter), allow_fail=False) + node.account.ssh("mv " + processor.LOG_FILE + " " + processor.LOG_FILE + roll_counter + str(counter), allow_fail=False) + + if new_version == str(DEV_VERSION): + processor.set_version("") # set to TRUNK + else: + processor.set_version(new_version) + processor.set_upgrade_from(upgrade_from) + + grep_metadata_error = "grep \"org.apache.kafka.streams.errors.TaskAssignmentException: unable to decode subscription data: version=2\" " + with node.account.monitor_log(processor.STDOUT_FILE) as monitor: + with node.account.monitor_log(processor.LOG_FILE) as log_monitor: + with first_other_node.account.monitor_log(first_other_processor.STDOUT_FILE) as first_other_monitor: + with second_other_node.account.monitor_log(second_other_processor.STDOUT_FILE) as second_other_monitor: + processor.start() + + log_monitor.wait_until("Kafka version : " + new_version, + timeout_sec=60, + err_msg="Could not detect Kafka Streams version " + new_version + " " + str(node.account)) + first_other_monitor.wait_until("processed 100 records from topic", + timeout_sec=60, + err_msg="Never saw output 'processed 100 records from topic' on" + str(first_other_node.account)) + found = list(first_other_node.account.ssh_capture(grep_metadata_error + first_other_processor.STDERR_FILE, allow_fail=True)) + if len(found) > 0: + raise Exception("Kafka Streams failed with 'unable to decode subscription data: version=2'") + + second_other_monitor.wait_until("processed 100 records from topic", + timeout_sec=60, + err_msg="Never saw output 'processed 100 records from topic' on" + str(second_other_node.account)) + found = list(second_other_node.account.ssh_capture(grep_metadata_error + second_other_processor.STDERR_FILE, allow_fail=True)) + if len(found) > 0: + raise Exception("Kafka Streams failed with 'unable to decode subscription data: version=2'") + + monitor.wait_until("processed 100 records from topic", + timeout_sec=60, + err_msg="Never saw output 'processed 100 records from topic' on" + str(node.account)) diff --git a/tests/kafkatest/version.py b/tests/kafkatest/version.py index f63a7c17ecd34..ee3f8b57ef0bf 100644 --- a/tests/kafkatest/version.py +++ b/tests/kafkatest/version.py @@ -61,6 +61,7 @@ def get_version(node=None): return DEV_BRANCH DEV_BRANCH = KafkaVersion("dev") +DEV_VERSION = KafkaVersion("1.1.1-SNAPSHOT") # 0.8.2.X versions V_0_8_2_1 = KafkaVersion("0.8.2.1") @@ -89,7 +90,14 @@ def get_version(node=None): LATEST_0_10 = LATEST_0_10_2 -# 0.11.0.0 versions +# 0.11.0.x versions V_0_11_0_0 = KafkaVersion("0.11.0.0") -LATEST_0_11_0 = V_0_11_0_0 +V_0_11_0_1 = KafkaVersion("0.11.0.1") +V_0_11_0_2 = KafkaVersion("0.11.0.2") +LATEST_0_11_0 = V_0_11_0_2 LATEST_0_11 = LATEST_0_11_0 + +# 1.0.x versions +V_1_0_0 = KafkaVersion("1.0.0") +V_1_0_1 = KafkaVersion("1.0.1") +LATEST_1_0 = V_1_0_1 \ No newline at end of file diff --git a/vagrant/base.sh b/vagrant/base.sh index 4b5540652faa8..c520d49971ff9 100755 --- a/vagrant/base.sh +++ b/vagrant/base.sh @@ -99,8 +99,10 @@ popd popd popd -# Test multiple Scala versions -get_kafka 0.8.2.2 2.10 +# Test multiple Kafka versions +# we want to use the latest Scala version per Kafka version +# however, we cannot pull in Scala 2.12 builds atm, because Scala 2.12 requires Java 8, but we use Java 7 to run the system tests +get_kafka 0.8.2.2 2.11 chmod a+rw /opt/kafka-0.8.2.2 get_kafka 0.9.0.1 2.11 chmod a+rw /opt/kafka-0.9.0.1 @@ -110,8 +112,10 @@ get_kafka 0.10.1.1 2.11 chmod a+rw /opt/kafka-0.10.1.1 get_kafka 0.10.2.1 2.11 chmod a+rw /opt/kafka-0.10.2.1 -get_kafka 0.11.0.0 2.11 -chmod a+rw /opt/kafka-0.11.0.0 +get_kafka 0.11.0.2 2.11 +chmod a+rw /opt/kafka-0.11.0.2 +get_kafka 1.0.1 2.11 +chmod a+rw /opt/kafka-1.0.1 # For EC2 nodes, we want to use /mnt, which should have the local disk. On local