cadonna commented on code in PR #12161: URL: https://github.com/apache/kafka/pull/12161#discussion_r893379903
########## streams/src/test/java/org/apache/kafka/streams/integration/PauseResumeIntegrationTest.java: ########## @@ -0,0 +1,380 @@ +/* + * 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.integration; + +import static java.util.Arrays.asList; +import static java.util.Collections.singletonList; +import static org.apache.kafka.streams.KeyValue.pair; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.cleanStateBeforeTest; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.getTopicSize; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForApplicationState; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitUntilStreamsHasPolled; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.time.Duration; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Properties; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.serialization.LongDeserializer; +import org.apache.kafka.common.serialization.LongSerializer; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KafkaStreams.State; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.LagInfo; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; +import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; +import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.streams.processor.internals.namedtopology.KafkaStreamsNamedTopologyWrapper; +import org.apache.kafka.streams.processor.internals.namedtopology.NamedTopologyBuilder; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.test.IntegrationTest; +import org.apache.kafka.test.TestUtils; +import org.hamcrest.CoreMatchers; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; + +@Category({IntegrationTest.class}) +public class PauseResumeIntegrationTest { + private static final Duration STARTUP_TIMEOUT = Duration.ofSeconds(45); + public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1); + private static Properties producerConfig; + private static Properties consumerConfig; + + private static final Materialized<Object, Long, KeyValueStore<Bytes, byte[]>> IN_MEMORY_STORE = + Materialized.as(Stores.inMemoryKeyValueStore("store")); + + private static final String INPUT_STREAM_1 = "input-stream-1"; + private static final String INPUT_STREAM_2 = "input-stream-2"; + private static final String OUTPUT_STREAM_1 = "output-stream-1"; + private static final String OUTPUT_STREAM_2 = "output-stream-2"; + private static final String TOPOLOGY1 = "topology1"; + private static final String TOPOLOGY2 = "topology2"; + + private static final List<KeyValue<String, Long>> STANDARD_INPUT_DATA = + asList(pair("A", 100L), pair("B", 200L), pair("A", 300L), pair("C", 400L), pair("C", -50L)); + private static final List<KeyValue<String, Long>> COUNT_OUTPUT_DATA = + asList(pair("B", 1L), pair("A", 2L), pair("C", 2L)); + private static final List<KeyValue<String, Long>> COUNT_OUTPUT_DATA2 = + asList(pair("B", 2L), pair("A", 4L), pair("C", 4L)); + + private String appId; + private KafkaStreams kafkaStreams, kafkaStreams2; + private KafkaStreamsNamedTopologyWrapper streamsNamedTopologyWrapper; + + @Rule + public final TestName testName = new TestName(); + + @BeforeClass + public static void startCluster() throws Exception { + CLUSTER.start(); + producerConfig = TestUtils.producerConfig(CLUSTER.bootstrapServers(), + StringSerializer.class, LongSerializer.class); + consumerConfig = TestUtils.consumerConfig(CLUSTER.bootstrapServers(), + StringDeserializer.class, LongDeserializer.class); + } + + @AfterClass + public static void closeCluster() { + CLUSTER.stop(); + } + + @Before + public void createTopics() throws InterruptedException { + cleanStateBeforeTest(CLUSTER, 2, INPUT_STREAM_1, INPUT_STREAM_2, OUTPUT_STREAM_1, OUTPUT_STREAM_2); + appId = safeUniqueTestName(PauseResumeIntegrationTest.class, testName); + } + + private Properties props() { + final Properties properties = new Properties(); + properties.put(StreamsConfig.APPLICATION_ID_CONFIG, appId); + properties.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); + properties.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath()); + properties.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + properties.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Long().getClass()); + properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L); + properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + properties.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, StreamsConfig.OPTIMIZE); + properties.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 100); + properties.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000); + return properties; + } + + @After + public void shutdown() throws InterruptedException { + for (final KafkaStreams streams : Arrays.asList(kafkaStreams, kafkaStreams2, streamsNamedTopologyWrapper)) { + if (streams != null) { + streams.close(Duration.ofSeconds(30)); + } + } + } + + private static void produceToInputTopics(final String topic, final Collection<KeyValue<String, Long>> records) { + IntegrationTestUtils.produceKeyValuesSynchronously(topic, records, producerConfig, CLUSTER.time); + } + + @Test + public void shouldPauseAndResumeKafkaStreams() throws Exception { + kafkaStreams = buildKafkaStreams(OUTPUT_STREAM_1); + kafkaStreams.start(); + waitForApplicationState(singletonList(kafkaStreams), State.RUNNING, STARTUP_TIMEOUT); + + produceToInputTopics(INPUT_STREAM_1, STANDARD_INPUT_DATA); + awaitOutput(OUTPUT_STREAM_1, 3, COUNT_OUTPUT_DATA); + + kafkaStreams.pause(); + assertTrue(kafkaStreams.isPaused()); + + produceToInputTopics(INPUT_STREAM_1, STANDARD_INPUT_DATA); + assertNoLag(kafkaStreams); + + waitUntilStreamsHasPolled(kafkaStreams, 2); + assertTopicSize(OUTPUT_STREAM_1, 3); + + kafkaStreams.resume(); + assertFalse(kafkaStreams.isPaused()); + + awaitOutput(OUTPUT_STREAM_1, 3, COUNT_OUTPUT_DATA2); + assertTopicSize(OUTPUT_STREAM_1, 6); + } + + @Test + public void shouldAllowForTopologiesToStartPaused() throws Exception { + kafkaStreams = buildKafkaStreams(OUTPUT_STREAM_1); + kafkaStreams.pause(); + kafkaStreams.start(); + waitForApplicationState(singletonList(kafkaStreams), State.RUNNING, STARTUP_TIMEOUT); + assertTrue(kafkaStreams.isPaused()); + + produceToInputTopics(INPUT_STREAM_1, STANDARD_INPUT_DATA); + assertNoLag(kafkaStreams); + + waitUntilStreamsHasPolled(kafkaStreams, 2); + assertTopicSize(OUTPUT_STREAM_1, 0); + + kafkaStreams.resume(); + assertFalse(kafkaStreams.isPaused()); + awaitOutput(OUTPUT_STREAM_1, 3, COUNT_OUTPUT_DATA); + assertTopicSize(OUTPUT_STREAM_1, 3); + } + + @Test + public void shouldPauseAndResumeKafkaStreamsWithNamedTopologies() throws Exception { + streamsNamedTopologyWrapper = new KafkaStreamsNamedTopologyWrapper(props()); + final NamedTopologyBuilder builder1 = getNamedTopologyBuilder1(); + final NamedTopologyBuilder builder2 = getNamedTopologyBuilder2(); + + streamsNamedTopologyWrapper.start(asList(builder1.build(), builder2.build())); + waitForApplicationState(singletonList(streamsNamedTopologyWrapper), State.RUNNING, STARTUP_TIMEOUT); + + produceToInputTopics(INPUT_STREAM_1, STANDARD_INPUT_DATA); + produceToInputTopics(INPUT_STREAM_2, STANDARD_INPUT_DATA); + + awaitOutput(OUTPUT_STREAM_1, 3, COUNT_OUTPUT_DATA); + awaitOutput(OUTPUT_STREAM_2, 3, COUNT_OUTPUT_DATA); + assertTopicSize(OUTPUT_STREAM_1, 3); + assertTopicSize(OUTPUT_STREAM_2, 3); + + streamsNamedTopologyWrapper.pauseNamedTopology(TOPOLOGY1); + assertTrue(streamsNamedTopologyWrapper.isNamedTopologyPaused(TOPOLOGY1)); + assertFalse(streamsNamedTopologyWrapper.isNamedTopologyPaused(TOPOLOGY2)); + assertFalse(streamsNamedTopologyWrapper.isPaused()); + + produceToInputTopics(INPUT_STREAM_1, STANDARD_INPUT_DATA); + produceToInputTopics(INPUT_STREAM_2, STANDARD_INPUT_DATA); + + assertNoLag(streamsNamedTopologyWrapper); + + awaitOutput(OUTPUT_STREAM_2, 3, COUNT_OUTPUT_DATA2); + assertTopicSize(OUTPUT_STREAM_1, 3); + assertTopicSize(OUTPUT_STREAM_2, 6); + + streamsNamedTopologyWrapper.resumeNamedTopology(TOPOLOGY1); + assertFalse(streamsNamedTopologyWrapper.isNamedTopologyPaused(TOPOLOGY1)); + awaitOutput(OUTPUT_STREAM_1, 3, COUNT_OUTPUT_DATA2); + } + + @Test + public void shouldPauseAndResumeAllKafkaStreamsWithNamedTopologies() throws Exception { + streamsNamedTopologyWrapper = new KafkaStreamsNamedTopologyWrapper(props()); + final NamedTopologyBuilder builder1 = getNamedTopologyBuilder1(); + final NamedTopologyBuilder builder2 = getNamedTopologyBuilder2(); + + streamsNamedTopologyWrapper.start(asList(builder1.build(), builder2.build())); + waitForApplicationState(singletonList(streamsNamedTopologyWrapper), State.RUNNING, STARTUP_TIMEOUT); + + produceToInputTopics(INPUT_STREAM_1, STANDARD_INPUT_DATA); + produceToInputTopics(INPUT_STREAM_2, STANDARD_INPUT_DATA); + awaitOutput(OUTPUT_STREAM_1, 3, COUNT_OUTPUT_DATA); + awaitOutput(OUTPUT_STREAM_2, 3, COUNT_OUTPUT_DATA); + + streamsNamedTopologyWrapper.pause(); + assertTrue(streamsNamedTopologyWrapper.isPaused()); + assertTrue(streamsNamedTopologyWrapper.isNamedTopologyPaused(TOPOLOGY1)); + assertTrue(streamsNamedTopologyWrapper.isNamedTopologyPaused(TOPOLOGY2)); + + produceToInputTopics(INPUT_STREAM_1, STANDARD_INPUT_DATA); + produceToInputTopics(INPUT_STREAM_2, STANDARD_INPUT_DATA); + + assertNoLag(streamsNamedTopologyWrapper); + + waitUntilStreamsHasPolled(streamsNamedTopologyWrapper, 2); + assertTopicSize(OUTPUT_STREAM_1, 3); + assertTopicSize(OUTPUT_STREAM_2, 3); + + streamsNamedTopologyWrapper.resumeNamedTopology(TOPOLOGY1); + assertFalse(streamsNamedTopologyWrapper.isPaused()); + assertFalse(streamsNamedTopologyWrapper.isNamedTopologyPaused(TOPOLOGY1)); + assertTrue(streamsNamedTopologyWrapper.isNamedTopologyPaused(TOPOLOGY2)); + awaitOutput(OUTPUT_STREAM_1, 3, COUNT_OUTPUT_DATA2); + assertTopicSize(OUTPUT_STREAM_1, 6); + assertTopicSize(OUTPUT_STREAM_2, 3); + } + + @Test + public void shouldAllowForNamedTopologiesToStartPaused() throws Exception { + streamsNamedTopologyWrapper = new KafkaStreamsNamedTopologyWrapper(props()); + final NamedTopologyBuilder builder1 = getNamedTopologyBuilder1(); + final NamedTopologyBuilder builder2 = getNamedTopologyBuilder2(); + + streamsNamedTopologyWrapper.pauseNamedTopology(TOPOLOGY1); + streamsNamedTopologyWrapper.start(asList(builder1.build(), builder2.build())); + waitForApplicationState(singletonList(streamsNamedTopologyWrapper), State.RUNNING, STARTUP_TIMEOUT); + + assertFalse(streamsNamedTopologyWrapper.isPaused()); + assertTrue(streamsNamedTopologyWrapper.isNamedTopologyPaused(TOPOLOGY1)); + assertFalse(streamsNamedTopologyWrapper.isNamedTopologyPaused(TOPOLOGY2)); + + produceToInputTopics(INPUT_STREAM_1, STANDARD_INPUT_DATA); + produceToInputTopics(INPUT_STREAM_2, STANDARD_INPUT_DATA); + + awaitOutput(OUTPUT_STREAM_2, 3, COUNT_OUTPUT_DATA); + assertTopicSize(OUTPUT_STREAM_1, 0); + + streamsNamedTopologyWrapper.pause(); + assertTrue(streamsNamedTopologyWrapper.isPaused()); + assertTrue(streamsNamedTopologyWrapper.isNamedTopologyPaused(TOPOLOGY1)); + assertTrue(streamsNamedTopologyWrapper.isNamedTopologyPaused(TOPOLOGY2)); + + produceToInputTopics(INPUT_STREAM_1, STANDARD_INPUT_DATA); + produceToInputTopics(INPUT_STREAM_2, STANDARD_INPUT_DATA); + + assertNoLag(streamsNamedTopologyWrapper); + + waitUntilStreamsHasPolled(streamsNamedTopologyWrapper, 2); + assertTopicSize(OUTPUT_STREAM_1, 0); + assertTopicSize(OUTPUT_STREAM_2, 3); + + streamsNamedTopologyWrapper.resumeNamedTopology(TOPOLOGY1); + assertFalse(streamsNamedTopologyWrapper.isPaused()); + assertFalse(streamsNamedTopologyWrapper.isNamedTopologyPaused(TOPOLOGY1)); + assertTrue(streamsNamedTopologyWrapper.isNamedTopologyPaused(TOPOLOGY2)); + + awaitOutput(OUTPUT_STREAM_1, 3, COUNT_OUTPUT_DATA2); + assertTopicSize(OUTPUT_STREAM_1, 3); + assertTopicSize(OUTPUT_STREAM_2, 3); + } + + @Test + public void pauseResumehouldWorkAcrossInstances() throws Exception { Review Comment: According to the builds, this test seems flaky. I think the reason is that the cache (in Streams, not the RocksDB cache) of the state stores is not set to zero. When the cache is not set to zero, the number of results that are send downstreams is not deterministic, because some intermediate results might be sent downstream and some not. I tried to set the cache to zero and was able to run 100+ times the test in a row without failure whereas with the cache > 0, the test failed much earlier. Note that by changing the cache size also the results that you verify change. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
