You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2022/05/13 20:15:36 UTC

[GitHub] [kafka] wcarlson5 commented on a diff in pull request #12161: DRAFT: KAFKA-13873 Add ability to Pause / Resume KafkaStreams Topologies

wcarlson5 commented on code in PR #12161:
URL: https://github.com/apache/kafka/pull/12161#discussion_r872731431


##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java:
##########
@@ -463,7 +463,10 @@ public void restore(final Map<TaskId, Task> tasks) {
                 final TaskId taskId = changelogs.get(partition).stateManager.taskId();
                 try {
                     if (restoreChangelog(changelogs.get(partition))) {
-                        tasks.get(taskId).clearTaskTimeout();
+                        final Task task = tasks.get(taskId);
+                        if (task != null) {

Review Comment:
   Why are we getting null here when we were not before?



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java:
##########
@@ -273,6 +273,12 @@ Collection<Task> allTasks() {
         return readOnlyTasks;
     }
 
+    Collection<Task> notPausedTasks() {
+        return Collections.unmodifiableCollection(readOnlyActiveTasks.stream().filter(t ->

Review Comment:
   nit: Can we align each call on its own line for readability?



##########
streams/src/test/java/org/apache/kafka/streams/integration/PauseResumeIntegrationTest.java:
##########
@@ -0,0 +1,897 @@
+/*
+ * 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.purgeLocalStreamsState;
+import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName;
+import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.startApplicationAndWaitUntilRunning;
+import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForApplicationState;
+import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForCompletion;
+import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForStandbyCompletion;
+import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.IsEqual.equalTo;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+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 java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.IntegerDeserializer;
+import org.apache.kafka.common.serialization.IntegerSerializer;
+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.common.utils.MockTime;
+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.Topology;
+import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
+import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
+import org.apache.kafka.streams.integration.utils.IntegrationTestUtils.TrackingStateRestoreListener;
+import org.apache.kafka.streams.kstream.Consumed;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Materialized;
+import org.apache.kafka.streams.processor.StateRestoreListener;
+import org.apache.kafka.streams.processor.TaskId;
+import org.apache.kafka.streams.processor.api.Processor;
+import org.apache.kafka.streams.processor.api.ProcessorContext;
+import org.apache.kafka.streams.processor.api.Record;
+import org.apache.kafka.streams.processor.internals.StateDirectory;
+import org.apache.kafka.streams.processor.internals.namedtopology.KafkaStreamsNamedTopologyWrapper;
+import org.apache.kafka.streams.processor.internals.namedtopology.NamedTopologyBuilder;
+import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.kafka.streams.state.StoreBuilder;
+import org.apache.kafka.streams.state.Stores;
+import org.apache.kafka.streams.state.internals.InMemoryKeyValueStore;
+import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder;
+import org.apache.kafka.streams.state.internals.OffsetCheckpoint;
+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"));

Review Comment:
   nit: line length



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java:
##########
@@ -897,7 +897,8 @@ private void initializeAndRestorePhase() {
         }
         // we can always let changelog reader try restoring in order to initialize the changelogs;
         // if there's no active restoring or standby updating it would not try to fetch any data
-        changelogReader.restore(taskManager.tasks());
+        // After KAFKA-13873, we only restore the not paused tasks.
+        changelogReader.restore(taskManager.notPausedTasks());

Review Comment:
   when a paused task is resumed will restore be called if necessary?



##########
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()));
+
+        // Resume KafkaStreams/topology
+        kafkaStreams.resume();
+        assertFalse(kafkaStreams.isPaused());
+
+        // Verify that the input written during the pause is processed.
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA2));
+    }
+
+    @Test
+    public void shouldAllowForTopologiesToStartPaused() throws Exception {
+        CLUSTER.deleteTopicsAndWait(EXISTING_STREAM);
+        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);

Review Comment:
   nit: clean up?



##########
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()));
+
+        // Resume KafkaStreams/topology
+        kafkaStreams.resume();
+        assertFalse(kafkaStreams.isPaused());
+
+        // Verify that the input written during the pause is processed.
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA2));
+    }
+
+    @Test
+    public void shouldAllowForTopologiesToStartPaused() throws Exception {
+        CLUSTER.deleteTopicsAndWait(EXISTING_STREAM);
+        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.pause(); // This pause processing.
+        kafkaStreams.start();
+        assertTrue(startupLatch.await(30, TimeUnit.SECONDS));
+
+        assertTrue(kafkaStreams.isPaused());
+
+        // Write data
+        produceToInputTopics(EXISTING_STREAM, STANDARD_INPUT_DATA);
+
+        // Verify no output
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 0), CoreMatchers.equalTo(Collections.emptyList()));
+
+        // 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()));
+
+        // Resume KafkaStreams/topology
+        kafkaStreams.resume();
+        assertFalse(kafkaStreams.isPaused());
+
+        // Verify that the input written during the pause is processed.
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA2));
+    }
+
+    @Test
+    public void shouldPauseAndResumeKafkaStreamsWithNamedTopologies() throws Exception {
+        CLUSTER.deleteTopicsAndWait(EXISTING_STREAM);
+        CLUSTER.deleteTopicsAndWait(NEW_STREAM);
+        CLUSTER.createTopic(EXISTING_STREAM, 2, 1);
+        CLUSTER.createTopic(NEW_STREAM, 2, 1);
+
+        // Create KafkaStream / NamedTopologyBuilders
+        final KafkaStreamsNamedTopologyWrapper streams = new KafkaStreamsNamedTopologyWrapper(props());

Review Comment:
   I don't see where this gets cleaned up...



##########
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));

Review Comment:
   you can probably just call close()



##########
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:
   Maybe you can wait the max poll interval?



##########
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()));
+
+        // Resume KafkaStreams/topology
+        kafkaStreams.resume();
+        assertFalse(kafkaStreams.isPaused());
+
+        // Verify that the input written during the pause is processed.
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA2));
+    }
+
+    @Test
+    public void shouldAllowForTopologiesToStartPaused() throws Exception {
+        CLUSTER.deleteTopicsAndWait(EXISTING_STREAM);
+        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.pause(); // This pause processing.
+        kafkaStreams.start();
+        assertTrue(startupLatch.await(30, TimeUnit.SECONDS));
+
+        assertTrue(kafkaStreams.isPaused());
+
+        // Write data
+        produceToInputTopics(EXISTING_STREAM, STANDARD_INPUT_DATA);
+
+        // Verify no output
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 0), CoreMatchers.equalTo(Collections.emptyList()));
+
+        // 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()));
+
+        // Resume KafkaStreams/topology
+        kafkaStreams.resume();
+        assertFalse(kafkaStreams.isPaused());
+
+        // Verify that the input written during the pause is processed.
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA2));
+    }
+
+    @Test
+    public void shouldPauseAndResumeKafkaStreamsWithNamedTopologies() throws Exception {
+        CLUSTER.deleteTopicsAndWait(EXISTING_STREAM);
+        CLUSTER.deleteTopicsAndWait(NEW_STREAM);
+        CLUSTER.createTopic(EXISTING_STREAM, 2, 1);
+        CLUSTER.createTopic(NEW_STREAM, 2, 1);
+
+        // Create KafkaStream / NamedTopologyBuilders
+        final KafkaStreamsNamedTopologyWrapper streams = new KafkaStreamsNamedTopologyWrapper(props());
+        final NamedTopologyBuilder builder1 = streams.newNamedTopologyBuilder("topology1");
+        builder1.stream(EXISTING_STREAM).groupByKey().count().toStream().to(OUTPUT_STREAM_1);
+
+        final NamedTopologyBuilder builder2 = streams.newNamedTopologyBuilder("topology2");
+        builder2.stream(NEW_STREAM).groupBy((k, v) -> k).count(IN_MEMORY_STORE).toStream().to(OUTPUT_STREAM_2);
+
+        // Start KafkaStream
+        streams.start(asList(builder1.build(), builder2.build()));
+        waitForApplicationState(singletonList(streams), State.RUNNING, STARTUP_TIMEOUT);
+
+        // Write data for topology1
+        produceToInputTopics(EXISTING_STREAM, STANDARD_INPUT_DATA);
+        produceToInputTopics(NEW_STREAM, STANDARD_INPUT_DATA);
+
+        // Verify output
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA));
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_2, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA));
+
+        // Pause KafkaStreams/topplogy1
+        // JNH: Pause topology1
+        streams.pauseNamedTopology("topology1");
+
+        // Use statelistener to verify transition to paused transition?
+        // Read status as Paused?
+        // JNH: Assert the topology1 is paused and topology2 is not paused
+        assertTrue(streams.isNamedTopologyPaused("topology1"));
+
+        // Write more data for topology 1 and topology 2
+        produceToInputTopics(EXISTING_STREAM, STANDARD_INPUT_DATA);
+        produceToInputTopics(NEW_STREAM, STANDARD_INPUT_DATA);
+
+        // Verify that consumer read new data -- AKA, there is no lag.
+        final Map<String, Map<Integer, LagInfo>> lagMap = streams.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 for topology1 somehow? (Hard to prove negative)
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 0), CoreMatchers.equalTo(Collections.emptyList()));
+
+        // Verify that topology 2 is not paused and has processed data.
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_2, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA2));
+
+
+        // Resume KafkaStreams/topology
+        // JNH: resume topology 1; assert that it is running.
+        streams.resumeNamedTopology("topology1");
+        assertFalse(streams.isNamedTopologyPaused("topoology1"));
+
+        // Verify that the input written during the pause is processed.
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA2));
+
+        // JNH: Assert any final notes.
+    }
+
+    @Test
+    public void shouldPauseAndResumeAllKafkaStreamsWithNamedTopologies() throws Exception {
+        CLUSTER.deleteTopicsAndWait(EXISTING_STREAM);
+        CLUSTER.deleteTopicsAndWait(NEW_STREAM);
+        CLUSTER.createTopic(EXISTING_STREAM, 2, 1);
+        CLUSTER.createTopic(NEW_STREAM, 2, 1);
+
+        // Create KafkaStream / NamedTopologyBuilders
+        final KafkaStreamsNamedTopologyWrapper streams = new KafkaStreamsNamedTopologyWrapper(props());
+        final NamedTopologyBuilder builder1 = streams.newNamedTopologyBuilder("topology1");
+        builder1.stream(EXISTING_STREAM).groupByKey().count().toStream().to(OUTPUT_STREAM_1);
+
+        final NamedTopologyBuilder builder2 = streams.newNamedTopologyBuilder("topology2");
+        builder2.stream(NEW_STREAM).groupBy((k, v) -> k).count(IN_MEMORY_STORE).toStream().to(OUTPUT_STREAM_2);
+
+        // Start KafkaStream
+        streams.start(asList(builder1.build(), builder2.build()));
+        waitForApplicationState(singletonList(streams), State.RUNNING, STARTUP_TIMEOUT);
+
+        // Write data for topology1
+        produceToInputTopics(EXISTING_STREAM, STANDARD_INPUT_DATA);
+        produceToInputTopics(NEW_STREAM, STANDARD_INPUT_DATA);
+
+        // Verify output
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA));
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_2, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA));
+
+        // Pause KafkaStreams/topplogy1
+        // JNH: Pause topology1
+        streams.pause();
+        //streams.pauseNamedTopology("topology1");
+
+        // Use statelistener to verify transition to paused transition?
+        // Read status as Paused?
+        // JNH: Assert the topology1 is paused and topology2 is not paused
+        assertTrue(streams.isPaused());
+        assertTrue(streams.isNamedTopologyPaused("topology1"));
+        assertTrue(streams.isNamedTopologyPaused("topology2"));
+
+
+        // Write more data for topology 1 and topology 2
+        produceToInputTopics(EXISTING_STREAM, STANDARD_INPUT_DATA);
+        produceToInputTopics(NEW_STREAM, STANDARD_INPUT_DATA);
+
+        // Verify that consumer read new data -- AKA, there is no lag.
+        final Map<String, Map<Integer, LagInfo>> lagMap = streams.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 for topology1 somehow? (Hard to prove negative)
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 0), CoreMatchers.equalTo(Collections.emptyList()));
+
+        // Verify that topology 2 is paused and has processed data.
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_2, 0), CoreMatchers.equalTo(Collections.emptyList()));
+
+        // Verify that topology 2 is not paused and has processed data.
+        //assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_2, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA2));
+
+
+        // Resume KafkaStreams/topology
+        // JNH: resume topology 1; assert that it is running.
+        streams.resumeNamedTopology("topology1");
+        assertFalse(streams.isNamedTopologyPaused("topoology1"));
+        assertFalse(streams.isPaused()); // Are all paused?  No.
+        assertTrue(streams.isNamedTopologyPaused("topology2"));  // Still paused
+
+        // Verify that the input written during the pause is processed.
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA2));

Review Comment:
   topology2 as well



##########
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()));
+
+        // Resume KafkaStreams/topology
+        kafkaStreams.resume();
+        assertFalse(kafkaStreams.isPaused());
+
+        // Verify that the input written during the pause is processed.
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA2));
+    }
+
+    @Test
+    public void shouldAllowForTopologiesToStartPaused() throws Exception {
+        CLUSTER.deleteTopicsAndWait(EXISTING_STREAM);
+        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.pause(); // This pause processing.
+        kafkaStreams.start();
+        assertTrue(startupLatch.await(30, TimeUnit.SECONDS));
+
+        assertTrue(kafkaStreams.isPaused());
+
+        // Write data
+        produceToInputTopics(EXISTING_STREAM, STANDARD_INPUT_DATA);
+
+        // Verify no output
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 0), CoreMatchers.equalTo(Collections.emptyList()));
+
+        // 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()));
+
+        // Resume KafkaStreams/topology
+        kafkaStreams.resume();
+        assertFalse(kafkaStreams.isPaused());
+
+        // Verify that the input written during the pause is processed.
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA2));
+    }
+
+    @Test
+    public void shouldPauseAndResumeKafkaStreamsWithNamedTopologies() throws Exception {
+        CLUSTER.deleteTopicsAndWait(EXISTING_STREAM);
+        CLUSTER.deleteTopicsAndWait(NEW_STREAM);
+        CLUSTER.createTopic(EXISTING_STREAM, 2, 1);
+        CLUSTER.createTopic(NEW_STREAM, 2, 1);
+
+        // Create KafkaStream / NamedTopologyBuilders
+        final KafkaStreamsNamedTopologyWrapper streams = new KafkaStreamsNamedTopologyWrapper(props());
+        final NamedTopologyBuilder builder1 = streams.newNamedTopologyBuilder("topology1");
+        builder1.stream(EXISTING_STREAM).groupByKey().count().toStream().to(OUTPUT_STREAM_1);
+
+        final NamedTopologyBuilder builder2 = streams.newNamedTopologyBuilder("topology2");
+        builder2.stream(NEW_STREAM).groupBy((k, v) -> k).count(IN_MEMORY_STORE).toStream().to(OUTPUT_STREAM_2);
+
+        // Start KafkaStream
+        streams.start(asList(builder1.build(), builder2.build()));
+        waitForApplicationState(singletonList(streams), State.RUNNING, STARTUP_TIMEOUT);
+
+        // Write data for topology1
+        produceToInputTopics(EXISTING_STREAM, STANDARD_INPUT_DATA);
+        produceToInputTopics(NEW_STREAM, STANDARD_INPUT_DATA);
+
+        // Verify output
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA));
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_2, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA));
+
+        // Pause KafkaStreams/topplogy1
+        // JNH: Pause topology1
+        streams.pauseNamedTopology("topology1");
+
+        // Use statelistener to verify transition to paused transition?
+        // Read status as Paused?
+        // JNH: Assert the topology1 is paused and topology2 is not paused
+        assertTrue(streams.isNamedTopologyPaused("topology1"));

Review Comment:
   Make sure the other topology is not paused?



##########
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()));
+
+        // Resume KafkaStreams/topology
+        kafkaStreams.resume();
+        assertFalse(kafkaStreams.isPaused());
+
+        // Verify that the input written during the pause is processed.
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA2));
+    }
+
+    @Test
+    public void shouldAllowForTopologiesToStartPaused() throws Exception {
+        CLUSTER.deleteTopicsAndWait(EXISTING_STREAM);
+        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.pause(); // This pause processing.
+        kafkaStreams.start();
+        assertTrue(startupLatch.await(30, TimeUnit.SECONDS));
+
+        assertTrue(kafkaStreams.isPaused());
+
+        // Write data
+        produceToInputTopics(EXISTING_STREAM, STANDARD_INPUT_DATA);
+
+        // Verify no output
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 0), CoreMatchers.equalTo(Collections.emptyList()));
+
+        // 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()));
+
+        // Resume KafkaStreams/topology
+        kafkaStreams.resume();
+        assertFalse(kafkaStreams.isPaused());
+
+        // Verify that the input written during the pause is processed.
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA2));
+    }
+
+    @Test
+    public void shouldPauseAndResumeKafkaStreamsWithNamedTopologies() throws Exception {
+        CLUSTER.deleteTopicsAndWait(EXISTING_STREAM);
+        CLUSTER.deleteTopicsAndWait(NEW_STREAM);
+        CLUSTER.createTopic(EXISTING_STREAM, 2, 1);
+        CLUSTER.createTopic(NEW_STREAM, 2, 1);
+
+        // Create KafkaStream / NamedTopologyBuilders
+        final KafkaStreamsNamedTopologyWrapper streams = new KafkaStreamsNamedTopologyWrapper(props());
+        final NamedTopologyBuilder builder1 = streams.newNamedTopologyBuilder("topology1");
+        builder1.stream(EXISTING_STREAM).groupByKey().count().toStream().to(OUTPUT_STREAM_1);
+
+        final NamedTopologyBuilder builder2 = streams.newNamedTopologyBuilder("topology2");
+        builder2.stream(NEW_STREAM).groupBy((k, v) -> k).count(IN_MEMORY_STORE).toStream().to(OUTPUT_STREAM_2);
+
+        // Start KafkaStream
+        streams.start(asList(builder1.build(), builder2.build()));
+        waitForApplicationState(singletonList(streams), State.RUNNING, STARTUP_TIMEOUT);
+
+        // Write data for topology1
+        produceToInputTopics(EXISTING_STREAM, STANDARD_INPUT_DATA);
+        produceToInputTopics(NEW_STREAM, STANDARD_INPUT_DATA);
+
+        // Verify output
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA));
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_2, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA));
+
+        // Pause KafkaStreams/topplogy1
+        // JNH: Pause topology1
+        streams.pauseNamedTopology("topology1");
+
+        // Use statelistener to verify transition to paused transition?
+        // Read status as Paused?
+        // JNH: Assert the topology1 is paused and topology2 is not paused
+        assertTrue(streams.isNamedTopologyPaused("topology1"));
+
+        // Write more data for topology 1 and topology 2
+        produceToInputTopics(EXISTING_STREAM, STANDARD_INPUT_DATA);
+        produceToInputTopics(NEW_STREAM, STANDARD_INPUT_DATA);
+
+        // Verify that consumer read new data -- AKA, there is no lag.
+        final Map<String, Map<Integer, LagInfo>> lagMap = streams.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 for topology1 somehow? (Hard to prove negative)
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 0), CoreMatchers.equalTo(Collections.emptyList()));
+
+        // Verify that topology 2 is not paused and has processed data.
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_2, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA2));
+
+
+        // Resume KafkaStreams/topology
+        // JNH: resume topology 1; assert that it is running.
+        streams.resumeNamedTopology("topology1");
+        assertFalse(streams.isNamedTopologyPaused("topoology1"));
+
+        // Verify that the input written during the pause is processed.
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA2));
+
+        // JNH: Assert any final notes.
+    }
+
+    @Test
+    public void shouldPauseAndResumeAllKafkaStreamsWithNamedTopologies() throws Exception {
+        CLUSTER.deleteTopicsAndWait(EXISTING_STREAM);
+        CLUSTER.deleteTopicsAndWait(NEW_STREAM);
+        CLUSTER.createTopic(EXISTING_STREAM, 2, 1);
+        CLUSTER.createTopic(NEW_STREAM, 2, 1);
+
+        // Create KafkaStream / NamedTopologyBuilders
+        final KafkaStreamsNamedTopologyWrapper streams = new KafkaStreamsNamedTopologyWrapper(props());
+        final NamedTopologyBuilder builder1 = streams.newNamedTopologyBuilder("topology1");
+        builder1.stream(EXISTING_STREAM).groupByKey().count().toStream().to(OUTPUT_STREAM_1);
+
+        final NamedTopologyBuilder builder2 = streams.newNamedTopologyBuilder("topology2");
+        builder2.stream(NEW_STREAM).groupBy((k, v) -> k).count(IN_MEMORY_STORE).toStream().to(OUTPUT_STREAM_2);
+
+        // Start KafkaStream
+        streams.start(asList(builder1.build(), builder2.build()));
+        waitForApplicationState(singletonList(streams), State.RUNNING, STARTUP_TIMEOUT);
+
+        // Write data for topology1
+        produceToInputTopics(EXISTING_STREAM, STANDARD_INPUT_DATA);
+        produceToInputTopics(NEW_STREAM, STANDARD_INPUT_DATA);
+
+        // Verify output
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA));
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_2, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA));
+
+        // Pause KafkaStreams/topplogy1
+        // JNH: Pause topology1
+        streams.pause();
+        //streams.pauseNamedTopology("topology1");
+
+        // Use statelistener to verify transition to paused transition?
+        // Read status as Paused?
+        // JNH: Assert the topology1 is paused and topology2 is not paused
+        assertTrue(streams.isPaused());
+        assertTrue(streams.isNamedTopologyPaused("topology1"));
+        assertTrue(streams.isNamedTopologyPaused("topology2"));
+
+
+        // Write more data for topology 1 and topology 2
+        produceToInputTopics(EXISTING_STREAM, STANDARD_INPUT_DATA);
+        produceToInputTopics(NEW_STREAM, STANDARD_INPUT_DATA);
+
+        // Verify that consumer read new data -- AKA, there is no lag.
+        final Map<String, Map<Integer, LagInfo>> lagMap = streams.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 for topology1 somehow? (Hard to prove negative)
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 0), CoreMatchers.equalTo(Collections.emptyList()));
+
+        // Verify that topology 2 is paused and has processed data.
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_2, 0), CoreMatchers.equalTo(Collections.emptyList()));
+
+        // Verify that topology 2 is not paused and has processed data.

Review Comment:
   clean up



##########
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()));
+
+        // Resume KafkaStreams/topology
+        kafkaStreams.resume();
+        assertFalse(kafkaStreams.isPaused());
+
+        // Verify that the input written during the pause is processed.
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), CoreMatchers.equalTo(COUNT_OUTPUT_DATA2));
+    }
+
+    @Test
+    public void shouldAllowForTopologiesToStartPaused() throws Exception {
+        CLUSTER.deleteTopicsAndWait(EXISTING_STREAM);
+        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.pause(); // This pause processing.
+        kafkaStreams.start();
+        assertTrue(startupLatch.await(30, TimeUnit.SECONDS));
+
+        assertTrue(kafkaStreams.isPaused());
+
+        // Write data
+        produceToInputTopics(EXISTING_STREAM, STANDARD_INPUT_DATA);
+
+        // Verify no output
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 0), CoreMatchers.equalTo(Collections.emptyList()));
+
+        // Pause KafkaStreams/topplogy
+        kafkaStreams.pause();

Review Comment:
   do we need to pause twice?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org