jnh5y commented on code in PR #12161: URL: https://github.com/apache/kafka/pull/12161#discussion_r893450731
########## streams/src/test/java/org/apache/kafka/streams/integration/PauseResumeIntegrationTest.java: ########## @@ -0,0 +1,501 @@ +/* + * 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.safeUniqueTestName; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForApplicationState; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived; +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.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +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); + private static final int NUM_BROKERS = 1; + + public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS); + 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")); + + + // existing topic that is pre-filled but cleared between tests + private static final String EXISTING_STREAM = "existing-stream"; + private static final String NEW_STREAM = "new-stream"; + + private static final String INPUT_STREAM_1 = "input-stream-1"; + private static final String INPUT_STREAM_2 = "input-stream-2"; + private static final String INPUT_STREAM_3 = "input-stream-3"; + + private static final String OUTPUT_STREAM_1 = "output-stream-1"; + private static final String OUTPUT_STREAM_2 = "output-stream-2"; + private static final String OUTPUT_STREAM_3 = "output-stream-3"; + + 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)); + // JNH: Why are there 5 outputs unlike in NamedTopologyIT? + 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_DATA = + // asList(pair("A", 1L), pair("B", 1L), pair("A", 2L), pair("C", 1L), pair("C", 2L)); + private static final List<KeyValue<String, Long>> COUNT_OUTPUT_DATA2 = + asList(pair("B", 2L), pair("A", 4L), pair("C", 4L)); + // output of count operation with caching + // private static final List<KeyValue<String, Long>> COUNT_OUTPUT_DATA = + // asList(pair("B", 1L), pair("A", 2L), pair("C", 2L)); // output of count operation with caching + + @BeforeClass + public static void startCluster() throws Exception { + CLUSTER.start(); + + CLUSTER.createTopic(INPUT_STREAM_1, 2, 1); + CLUSTER.createTopic(INPUT_STREAM_2, 2, 1); + CLUSTER.createTopic(INPUT_STREAM_3, 2, 1); + + producerConfig = TestUtils.producerConfig(CLUSTER.bootstrapServers(), StringSerializer.class, LongSerializer.class); + consumerConfig = TestUtils.consumerConfig(CLUSTER.bootstrapServers(), StringDeserializer.class, LongDeserializer.class); + + CLUSTER.createTopic(OUTPUT_STREAM_1, 2, 1); + CLUSTER.createTopic(OUTPUT_STREAM_2, 2, 1); + CLUSTER.createTopic(OUTPUT_STREAM_3, 2, 1); + } + + @AfterClass + public static void closeCluster() { + CLUSTER.stop(); + } + + @Rule + public final TestName testName = new TestName(); + private String appId; + private KafkaStreams kafkaStreams; + + @Before + public void createTopics() throws InterruptedException { + appId = safeUniqueTestName(PauseResumeIntegrationTest.class, testName); + + final Properties adminConfig = new Properties(); + adminConfig.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); + } + + private Properties props() { + final Properties streamsConfiguration = new Properties(); + streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, appId); + streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); + //streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath()); + streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Long().getClass()); + streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L); + streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + return streamsConfiguration; + } + + @After + public void shutdown() { + if (kafkaStreams != null) { + kafkaStreams.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 { + + CLUSTER.createTopic(EXISTING_STREAM, 2, 1); + + // Create KafkaStream + final StreamsBuilder builder = new StreamsBuilder(); + //builder.stream(EXISTING_STREAM).groupBy((k, v) -> k).count(IN_MEMORY_STORE).toStream().to(OUTPUT_STREAM_1); + builder.stream(EXISTING_STREAM).groupByKey().count().toStream().to(OUTPUT_STREAM_1); + final Properties props = props(); + props.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, StreamsConfig.OPTIMIZE); + + kafkaStreams = new KafkaStreams(builder.build(props), props); + + // Shows pattern for blocking on state change. + final CountDownLatch startupLatch = new CountDownLatch(1); + kafkaStreams.setStateListener((newState, oldState) -> { + if (newState == State.RUNNING && oldState == State.REBALANCING) { + startupLatch.countDown(); + } + }); + + // Start KafkaStream + kafkaStreams.start(); + assertTrue(startupLatch.await(30, TimeUnit.SECONDS)); + + // Write data + produceToInputTopics(EXISTING_STREAM, STANDARD_INPUT_DATA); + + // Verify output + assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA)); + + // Pause KafkaStreams/topplogy + kafkaStreams.pause(); + + // Use statelistener to verify transition to paused transition? + // Read status as Paused? + assertTrue(kafkaStreams.isPaused()); + + // Write more data + produceToInputTopics(EXISTING_STREAM, STANDARD_INPUT_DATA); + + // Verify that consumer read new data -- AKA, there is no lag. + final Map<String, Map<Integer, LagInfo>> lagMap = kafkaStreams.allLocalStorePartitionLags(); + final Long maxLag = lagMap.values().stream().flatMap(m -> m.values().stream()).map(LagInfo::offsetLag).max(Long::compare).get(); + assertEquals(0, (long) maxLag); + + // Verify no output somehow? (Hard to prove negative) + assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 0), CoreMatchers.equalTo(Collections.emptyList())); Review Comment: After some work, we've found a slightly better way to work through this. -- 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: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org