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


##########
streams/src/test/java/org/apache/kafka/streams/integration/PauseResumeIntegrationTest.java:
##########
@@ -0,0 +1,445 @@
+/*
+ * 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 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"));
+
+    private static final String INPUT_STREAM_1 = "input-stream-1";
+    private static final String INPUT_STREAM_2 = "input-stream-2";
+
+    private static final String OUTPUT_STREAM_1 = "output-stream-1";
+    private static final String OUTPUT_STREAM_2 = "output-stream-2";
+    
+    private static final String TOPOLOGY1 = "topology1";
+    private static final String TOPOLOGY2 = "topology2";
+
+    private static final List<KeyValue<String, Long>> STANDARD_INPUT_DATA =
+        asList(pair("A", 100L), pair("B", 200L), pair("A", 300L), pair("C", 
400L), pair("C", -50L));
+    private static final List<KeyValue<String, Long>> COUNT_OUTPUT_DATA =
+        asList(pair("B", 1L), pair("A", 2L), pair("C", 2L));
+    private static final List<KeyValue<String, Long>> COUNT_OUTPUT_DATA2 =
+        asList(pair("B", 2L), pair("A", 4L), pair("C", 4L));
+
+    @BeforeClass
+    public static void startCluster() throws Exception {
+        CLUSTER.start();
+
+        producerConfig = TestUtils.producerConfig(CLUSTER.bootstrapServers(),
+            StringSerializer.class, LongSerializer.class);
+        consumerConfig = TestUtils.consumerConfig(CLUSTER.bootstrapServers(),
+            StringDeserializer.class, LongDeserializer.class);
+    }
+
+    @AfterClass
+    public static void closeCluster() {
+        CLUSTER.stop();
+    }
+
+    @Rule
+    public final TestName testName = new TestName();
+    private String appId;
+    private KafkaStreams kafkaStreams;
+    private KafkaStreamsNamedTopologyWrapper streamsNamedTopologyWrapper;
+
+    @Before
+    public void createTopics() throws InterruptedException {
+        appId = safeUniqueTestName(PauseResumeIntegrationTest.class, testName);
+
+        CLUSTER.createTopic(INPUT_STREAM_1, 2, 1);
+        CLUSTER.createTopic(INPUT_STREAM_2, 2, 1);
+        CLUSTER.createTopic(OUTPUT_STREAM_1, 2, 1);
+        CLUSTER.createTopic(OUTPUT_STREAM_2, 2, 1);
+    }
+
+    private Properties props() {
+        final Properties properties = new Properties();
+        properties.put(StreamsConfig.APPLICATION_ID_CONFIG, appId);
+        properties.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, 
CLUSTER.bootstrapServers());
+        properties.put(StreamsConfig.STATE_DIR_CONFIG,
+            TestUtils.tempDirectory(appId).getPath());
+        properties.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, 
Serdes.String().getClass());
+        properties.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, 
Serdes.Long().getClass());
+        properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
+        properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
+        properties.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, 
StreamsConfig.OPTIMIZE);
+        return properties;
+    }
+
+    @After
+    public void shutdown() throws InterruptedException {
+        if (kafkaStreams != null) {
+            kafkaStreams.close(Duration.ofSeconds(30));
+        }
+        if (streamsNamedTopologyWrapper != null) {
+            streamsNamedTopologyWrapper.close(Duration.ofSeconds(30));
+        }
+        CLUSTER.deleteTopicsAndWait(INPUT_STREAM_1);
+        CLUSTER.deleteTopicsAndWait(INPUT_STREAM_2);
+        CLUSTER.deleteTopicsAndWait(OUTPUT_STREAM_1);
+        CLUSTER.deleteTopicsAndWait(OUTPUT_STREAM_2);
+    }
+
+    private static void produceToInputTopics(final String topic,
+        final Collection<KeyValue<String, Long>> records) {

Review Comment:
   nit:
   ```suggestion
       private static void produceToInputTopics(final String topic, final 
Collection<KeyValue<String, Long>> records) {
   ```



##########
streams/src/test/java/org/apache/kafka/streams/integration/PauseResumeIntegrationTest.java:
##########
@@ -0,0 +1,445 @@
+/*
+ * 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 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"));
+
+    private static final String INPUT_STREAM_1 = "input-stream-1";
+    private static final String INPUT_STREAM_2 = "input-stream-2";
+
+    private static final String OUTPUT_STREAM_1 = "output-stream-1";
+    private static final String OUTPUT_STREAM_2 = "output-stream-2";
+    
+    private static final String TOPOLOGY1 = "topology1";
+    private static final String TOPOLOGY2 = "topology2";
+
+    private static final List<KeyValue<String, Long>> STANDARD_INPUT_DATA =
+        asList(pair("A", 100L), pair("B", 200L), pair("A", 300L), pair("C", 
400L), pair("C", -50L));
+    private static final List<KeyValue<String, Long>> COUNT_OUTPUT_DATA =
+        asList(pair("B", 1L), pair("A", 2L), pair("C", 2L));
+    private static final List<KeyValue<String, Long>> COUNT_OUTPUT_DATA2 =
+        asList(pair("B", 2L), pair("A", 4L), pair("C", 4L));
+
+    @BeforeClass
+    public static void startCluster() throws Exception {
+        CLUSTER.start();
+
+        producerConfig = TestUtils.producerConfig(CLUSTER.bootstrapServers(),
+            StringSerializer.class, LongSerializer.class);
+        consumerConfig = TestUtils.consumerConfig(CLUSTER.bootstrapServers(),
+            StringDeserializer.class, LongDeserializer.class);
+    }
+
+    @AfterClass
+    public static void closeCluster() {
+        CLUSTER.stop();
+    }
+
+    @Rule
+    public final TestName testName = new TestName();
+    private String appId;
+    private KafkaStreams kafkaStreams;
+    private KafkaStreamsNamedTopologyWrapper streamsNamedTopologyWrapper;
+
+    @Before
+    public void createTopics() throws InterruptedException {
+        appId = safeUniqueTestName(PauseResumeIntegrationTest.class, testName);
+
+        CLUSTER.createTopic(INPUT_STREAM_1, 2, 1);
+        CLUSTER.createTopic(INPUT_STREAM_2, 2, 1);
+        CLUSTER.createTopic(OUTPUT_STREAM_1, 2, 1);
+        CLUSTER.createTopic(OUTPUT_STREAM_2, 2, 1);
+    }
+
+    private Properties props() {
+        final Properties properties = new Properties();
+        properties.put(StreamsConfig.APPLICATION_ID_CONFIG, appId);
+        properties.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, 
CLUSTER.bootstrapServers());
+        properties.put(StreamsConfig.STATE_DIR_CONFIG,
+            TestUtils.tempDirectory(appId).getPath());
+        properties.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, 
Serdes.String().getClass());
+        properties.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, 
Serdes.Long().getClass());
+        properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
+        properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
+        properties.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, 
StreamsConfig.OPTIMIZE);
+        return properties;
+    }
+
+    @After
+    public void shutdown() throws InterruptedException {
+        if (kafkaStreams != null) {
+            kafkaStreams.close(Duration.ofSeconds(30));
+        }
+        if (streamsNamedTopologyWrapper != null) {
+            streamsNamedTopologyWrapper.close(Duration.ofSeconds(30));
+        }
+        CLUSTER.deleteTopicsAndWait(INPUT_STREAM_1);
+        CLUSTER.deleteTopicsAndWait(INPUT_STREAM_2);
+        CLUSTER.deleteTopicsAndWait(OUTPUT_STREAM_1);
+        CLUSTER.deleteTopicsAndWait(OUTPUT_STREAM_2);
+    }
+
+    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 {
+        // Create KafkaStreams instance
+        final StreamsBuilder builder = new StreamsBuilder();
+        
builder.stream(INPUT_STREAM_1).groupByKey().count().toStream().to(OUTPUT_STREAM_1);
+
+        kafkaStreams = new KafkaStreams(builder.build(props()), props());
+        kafkaStreams.start();
+        waitForApplicationState(singletonList(kafkaStreams), State.RUNNING, 
STARTUP_TIMEOUT);
+
+        // Write data
+        produceToInputTopics(INPUT_STREAM_1, STANDARD_INPUT_DATA);
+
+        // Verify output
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, 
OUTPUT_STREAM_1, 3),
+            CoreMatchers.equalTo(COUNT_OUTPUT_DATA));
+
+        // Pause KafkaStreams/topology
+        kafkaStreams.pause();
+        assertTrue(kafkaStreams.isPaused());
+
+        // Write more data
+        produceToInputTopics(INPUT_STREAM_1, STANDARD_INPUT_DATA);
+
+        // Verify that consumers read new data -- AKA, there is no lag.
+        final Map<String, Map<Integer, LagInfo>> lagMap = 
kafkaStreams.allLocalStorePartitionLags();
+        assertNoLag(lagMap);
+
+        // Verify no output somehow?
+        // Is there a better way to show this?
+        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 {
+        // Create KafkaStreams instance
+        final StreamsBuilder builder = new StreamsBuilder();
+        
builder.stream(INPUT_STREAM_1).groupByKey().count().toStream().to(OUTPUT_STREAM_1);
+
+        kafkaStreams = new KafkaStreams(builder.build(props()), props());
+
+        // Start KafkaStream with paused processing.
+        kafkaStreams.pause();
+        kafkaStreams.start();
+        // Check for rebalancing instead?

Review Comment:
   Why would you like to wait for `REBALANCING` instead of `RUNNING`?



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

Review Comment:
   Unit tests would still be great!



##########
streams/src/test/java/org/apache/kafka/streams/integration/PauseResumeIntegrationTest.java:
##########
@@ -0,0 +1,445 @@
+/*
+ * 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 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"));
+
+    private static final String INPUT_STREAM_1 = "input-stream-1";
+    private static final String INPUT_STREAM_2 = "input-stream-2";
+
+    private static final String OUTPUT_STREAM_1 = "output-stream-1";
+    private static final String OUTPUT_STREAM_2 = "output-stream-2";
+    
+    private static final String TOPOLOGY1 = "topology1";
+    private static final String TOPOLOGY2 = "topology2";
+
+    private static final List<KeyValue<String, Long>> STANDARD_INPUT_DATA =
+        asList(pair("A", 100L), pair("B", 200L), pair("A", 300L), pair("C", 
400L), pair("C", -50L));
+    private static final List<KeyValue<String, Long>> COUNT_OUTPUT_DATA =
+        asList(pair("B", 1L), pair("A", 2L), pair("C", 2L));
+    private static final List<KeyValue<String, Long>> COUNT_OUTPUT_DATA2 =
+        asList(pair("B", 2L), pair("A", 4L), pair("C", 4L));
+
+    @BeforeClass
+    public static void startCluster() throws Exception {
+        CLUSTER.start();
+
+        producerConfig = TestUtils.producerConfig(CLUSTER.bootstrapServers(),
+            StringSerializer.class, LongSerializer.class);
+        consumerConfig = TestUtils.consumerConfig(CLUSTER.bootstrapServers(),
+            StringDeserializer.class, LongDeserializer.class);
+    }
+
+    @AfterClass
+    public static void closeCluster() {
+        CLUSTER.stop();
+    }
+
+    @Rule
+    public final TestName testName = new TestName();
+    private String appId;
+    private KafkaStreams kafkaStreams;
+    private KafkaStreamsNamedTopologyWrapper streamsNamedTopologyWrapper;
+
+    @Before
+    public void createTopics() throws InterruptedException {
+        appId = safeUniqueTestName(PauseResumeIntegrationTest.class, testName);
+
+        CLUSTER.createTopic(INPUT_STREAM_1, 2, 1);
+        CLUSTER.createTopic(INPUT_STREAM_2, 2, 1);
+        CLUSTER.createTopic(OUTPUT_STREAM_1, 2, 1);
+        CLUSTER.createTopic(OUTPUT_STREAM_2, 2, 1);
+    }
+
+    private Properties props() {
+        final Properties properties = new Properties();
+        properties.put(StreamsConfig.APPLICATION_ID_CONFIG, appId);
+        properties.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, 
CLUSTER.bootstrapServers());
+        properties.put(StreamsConfig.STATE_DIR_CONFIG,
+            TestUtils.tempDirectory(appId).getPath());
+        properties.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, 
Serdes.String().getClass());
+        properties.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, 
Serdes.Long().getClass());
+        properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
+        properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
+        properties.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, 
StreamsConfig.OPTIMIZE);
+        return properties;
+    }
+
+    @After
+    public void shutdown() throws InterruptedException {
+        if (kafkaStreams != null) {
+            kafkaStreams.close(Duration.ofSeconds(30));
+        }
+        if (streamsNamedTopologyWrapper != null) {
+            streamsNamedTopologyWrapper.close(Duration.ofSeconds(30));
+        }
+        CLUSTER.deleteTopicsAndWait(INPUT_STREAM_1);
+        CLUSTER.deleteTopicsAndWait(INPUT_STREAM_2);
+        CLUSTER.deleteTopicsAndWait(OUTPUT_STREAM_1);
+        CLUSTER.deleteTopicsAndWait(OUTPUT_STREAM_2);
+    }
+
+    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 {
+        // Create KafkaStreams instance

Review Comment:
   This and the other comments are rather obvious. I would remove them.



##########
streams/src/test/java/org/apache/kafka/streams/integration/PauseResumeIntegrationTest.java:
##########
@@ -0,0 +1,445 @@
+/*
+ * 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 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"));
+
+    private static final String INPUT_STREAM_1 = "input-stream-1";
+    private static final String INPUT_STREAM_2 = "input-stream-2";
+
+    private static final String OUTPUT_STREAM_1 = "output-stream-1";
+    private static final String OUTPUT_STREAM_2 = "output-stream-2";
+    
+    private static final String TOPOLOGY1 = "topology1";
+    private static final String TOPOLOGY2 = "topology2";
+
+    private static final List<KeyValue<String, Long>> STANDARD_INPUT_DATA =
+        asList(pair("A", 100L), pair("B", 200L), pair("A", 300L), pair("C", 
400L), pair("C", -50L));
+    private static final List<KeyValue<String, Long>> COUNT_OUTPUT_DATA =
+        asList(pair("B", 1L), pair("A", 2L), pair("C", 2L));
+    private static final List<KeyValue<String, Long>> COUNT_OUTPUT_DATA2 =
+        asList(pair("B", 2L), pair("A", 4L), pair("C", 4L));
+
+    @BeforeClass
+    public static void startCluster() throws Exception {
+        CLUSTER.start();
+
+        producerConfig = TestUtils.producerConfig(CLUSTER.bootstrapServers(),
+            StringSerializer.class, LongSerializer.class);
+        consumerConfig = TestUtils.consumerConfig(CLUSTER.bootstrapServers(),
+            StringDeserializer.class, LongDeserializer.class);
+    }
+
+    @AfterClass
+    public static void closeCluster() {
+        CLUSTER.stop();
+    }
+
+    @Rule
+    public final TestName testName = new TestName();
+    private String appId;
+    private KafkaStreams kafkaStreams;
+    private KafkaStreamsNamedTopologyWrapper streamsNamedTopologyWrapper;
+
+    @Before
+    public void createTopics() throws InterruptedException {
+        appId = safeUniqueTestName(PauseResumeIntegrationTest.class, testName);
+
+        CLUSTER.createTopic(INPUT_STREAM_1, 2, 1);
+        CLUSTER.createTopic(INPUT_STREAM_2, 2, 1);
+        CLUSTER.createTopic(OUTPUT_STREAM_1, 2, 1);
+        CLUSTER.createTopic(OUTPUT_STREAM_2, 2, 1);
+    }
+
+    private Properties props() {
+        final Properties properties = new Properties();
+        properties.put(StreamsConfig.APPLICATION_ID_CONFIG, appId);
+        properties.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, 
CLUSTER.bootstrapServers());
+        properties.put(StreamsConfig.STATE_DIR_CONFIG,
+            TestUtils.tempDirectory(appId).getPath());
+        properties.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, 
Serdes.String().getClass());
+        properties.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, 
Serdes.Long().getClass());
+        properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
+        properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
+        properties.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, 
StreamsConfig.OPTIMIZE);
+        return properties;
+    }
+
+    @After
+    public void shutdown() throws InterruptedException {
+        if (kafkaStreams != null) {
+            kafkaStreams.close(Duration.ofSeconds(30));
+        }
+        if (streamsNamedTopologyWrapper != null) {
+            streamsNamedTopologyWrapper.close(Duration.ofSeconds(30));
+        }
+        CLUSTER.deleteTopicsAndWait(INPUT_STREAM_1);
+        CLUSTER.deleteTopicsAndWait(INPUT_STREAM_2);
+        CLUSTER.deleteTopicsAndWait(OUTPUT_STREAM_1);
+        CLUSTER.deleteTopicsAndWait(OUTPUT_STREAM_2);
+    }
+
+    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 {

Review Comment:
   Could you please try to extract common code to reusable methods?



##########
streams/src/test/java/org/apache/kafka/streams/integration/PauseResumeIntegrationTest.java:
##########
@@ -0,0 +1,445 @@
+/*
+ * 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 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"));
+
+    private static final String INPUT_STREAM_1 = "input-stream-1";
+    private static final String INPUT_STREAM_2 = "input-stream-2";
+
+    private static final String OUTPUT_STREAM_1 = "output-stream-1";
+    private static final String OUTPUT_STREAM_2 = "output-stream-2";
+    
+    private static final String TOPOLOGY1 = "topology1";
+    private static final String TOPOLOGY2 = "topology2";
+
+    private static final List<KeyValue<String, Long>> STANDARD_INPUT_DATA =
+        asList(pair("A", 100L), pair("B", 200L), pair("A", 300L), pair("C", 
400L), pair("C", -50L));
+    private static final List<KeyValue<String, Long>> COUNT_OUTPUT_DATA =
+        asList(pair("B", 1L), pair("A", 2L), pair("C", 2L));
+    private static final List<KeyValue<String, Long>> COUNT_OUTPUT_DATA2 =
+        asList(pair("B", 2L), pair("A", 4L), pair("C", 4L));
+
+    @BeforeClass
+    public static void startCluster() throws Exception {
+        CLUSTER.start();
+
+        producerConfig = TestUtils.producerConfig(CLUSTER.bootstrapServers(),
+            StringSerializer.class, LongSerializer.class);
+        consumerConfig = TestUtils.consumerConfig(CLUSTER.bootstrapServers(),
+            StringDeserializer.class, LongDeserializer.class);
+    }
+
+    @AfterClass
+    public static void closeCluster() {
+        CLUSTER.stop();
+    }
+
+    @Rule
+    public final TestName testName = new TestName();
+    private String appId;
+    private KafkaStreams kafkaStreams;
+    private KafkaStreamsNamedTopologyWrapper streamsNamedTopologyWrapper;
+
+    @Before
+    public void createTopics() throws InterruptedException {
+        appId = safeUniqueTestName(PauseResumeIntegrationTest.class, testName);
+
+        CLUSTER.createTopic(INPUT_STREAM_1, 2, 1);
+        CLUSTER.createTopic(INPUT_STREAM_2, 2, 1);
+        CLUSTER.createTopic(OUTPUT_STREAM_1, 2, 1);
+        CLUSTER.createTopic(OUTPUT_STREAM_2, 2, 1);
+    }
+
+    private Properties props() {
+        final Properties properties = new Properties();
+        properties.put(StreamsConfig.APPLICATION_ID_CONFIG, appId);
+        properties.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, 
CLUSTER.bootstrapServers());
+        properties.put(StreamsConfig.STATE_DIR_CONFIG,
+            TestUtils.tempDirectory(appId).getPath());
+        properties.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, 
Serdes.String().getClass());
+        properties.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, 
Serdes.Long().getClass());
+        properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
+        properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
+        properties.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, 
StreamsConfig.OPTIMIZE);
+        return properties;
+    }
+
+    @After
+    public void shutdown() throws InterruptedException {
+        if (kafkaStreams != null) {
+            kafkaStreams.close(Duration.ofSeconds(30));
+        }
+        if (streamsNamedTopologyWrapper != null) {
+            streamsNamedTopologyWrapper.close(Duration.ofSeconds(30));
+        }
+        CLUSTER.deleteTopicsAndWait(INPUT_STREAM_1);
+        CLUSTER.deleteTopicsAndWait(INPUT_STREAM_2);
+        CLUSTER.deleteTopicsAndWait(OUTPUT_STREAM_1);
+        CLUSTER.deleteTopicsAndWait(OUTPUT_STREAM_2);
+    }
+
+    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 {
+        // Create KafkaStreams instance
+        final StreamsBuilder builder = new StreamsBuilder();
+        
builder.stream(INPUT_STREAM_1).groupByKey().count().toStream().to(OUTPUT_STREAM_1);
+
+        kafkaStreams = new KafkaStreams(builder.build(props()), props());
+        kafkaStreams.start();
+        waitForApplicationState(singletonList(kafkaStreams), State.RUNNING, 
STARTUP_TIMEOUT);
+
+        // Write data
+        produceToInputTopics(INPUT_STREAM_1, STANDARD_INPUT_DATA);
+
+        // Verify output
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, 
OUTPUT_STREAM_1, 3),
+            CoreMatchers.equalTo(COUNT_OUTPUT_DATA));
+
+        // Pause KafkaStreams/topology
+        kafkaStreams.pause();
+        assertTrue(kafkaStreams.isPaused());
+
+        // Write more data
+        produceToInputTopics(INPUT_STREAM_1, STANDARD_INPUT_DATA);
+
+        // Verify that consumers read new data -- AKA, there is no lag.
+        final Map<String, Map<Integer, LagInfo>> lagMap = 
kafkaStreams.allLocalStorePartitionLags();
+        assertNoLag(lagMap);

Review Comment:
   Just to be clear, this works as long as the input buffers for polled records 
are not full. I think it is good enough for this test, though.



##########
streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java:
##########
@@ -783,6 +822,28 @@ public void shouldThrowOnCleanupWhileRunning() throws 
InterruptedException {
         }
     }
 
+    @Test
+    public void shouldThrowOnCleanupWhilePaused() throws InterruptedException {
+        try (final KafkaStreams streams = new 
KafkaStreams(getBuilderWithSource().build(), props, supplier, time)) {
+            streams.start();
+            waitForCondition(
+                () -> streams.state() == KafkaStreams.State.RUNNING,
+                "Streams never started.");
+
+            streams.pause();
+            waitForCondition(
+                streams::isPaused,
+                "Streams did not paused");
+
+            try {
+                streams.cleanUp();
+                fail("Should have thrown IllegalStateException");
+            } catch (final IllegalStateException expected) {
+                assertEquals("Cannot clean up while running.", 
expected.getMessage());
+            }

Review Comment:
   I would use `assertThrows()` in the new test. The existing codebase also 
uses it in some tests like 
`shouldThrowExceptionSettingUncaughtExceptionHandlerNotInCreateState`.
   I would open a new PR to refactor the tests that use `try-fail-catch`.  That 
is optional. 



##########
streams/src/test/java/org/apache/kafka/streams/integration/PauseResumeIntegrationTest.java:
##########
@@ -0,0 +1,445 @@
+/*
+ * 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 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"));
+
+    private static final String INPUT_STREAM_1 = "input-stream-1";
+    private static final String INPUT_STREAM_2 = "input-stream-2";
+
+    private static final String OUTPUT_STREAM_1 = "output-stream-1";
+    private static final String OUTPUT_STREAM_2 = "output-stream-2";
+    
+    private static final String TOPOLOGY1 = "topology1";
+    private static final String TOPOLOGY2 = "topology2";
+
+    private static final List<KeyValue<String, Long>> STANDARD_INPUT_DATA =
+        asList(pair("A", 100L), pair("B", 200L), pair("A", 300L), pair("C", 
400L), pair("C", -50L));
+    private static final List<KeyValue<String, Long>> COUNT_OUTPUT_DATA =
+        asList(pair("B", 1L), pair("A", 2L), pair("C", 2L));
+    private static final List<KeyValue<String, Long>> COUNT_OUTPUT_DATA2 =
+        asList(pair("B", 2L), pair("A", 4L), pair("C", 4L));
+
+    @BeforeClass
+    public static void startCluster() throws Exception {
+        CLUSTER.start();
+
+        producerConfig = TestUtils.producerConfig(CLUSTER.bootstrapServers(),
+            StringSerializer.class, LongSerializer.class);
+        consumerConfig = TestUtils.consumerConfig(CLUSTER.bootstrapServers(),
+            StringDeserializer.class, LongDeserializer.class);
+    }
+
+    @AfterClass
+    public static void closeCluster() {
+        CLUSTER.stop();
+    }
+
+    @Rule
+    public final TestName testName = new TestName();
+    private String appId;
+    private KafkaStreams kafkaStreams;
+    private KafkaStreamsNamedTopologyWrapper streamsNamedTopologyWrapper;
+
+    @Before
+    public void createTopics() throws InterruptedException {
+        appId = safeUniqueTestName(PauseResumeIntegrationTest.class, testName);
+
+        CLUSTER.createTopic(INPUT_STREAM_1, 2, 1);
+        CLUSTER.createTopic(INPUT_STREAM_2, 2, 1);
+        CLUSTER.createTopic(OUTPUT_STREAM_1, 2, 1);
+        CLUSTER.createTopic(OUTPUT_STREAM_2, 2, 1);
+    }
+
+    private Properties props() {
+        final Properties properties = new Properties();
+        properties.put(StreamsConfig.APPLICATION_ID_CONFIG, appId);
+        properties.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, 
CLUSTER.bootstrapServers());
+        properties.put(StreamsConfig.STATE_DIR_CONFIG,
+            TestUtils.tempDirectory(appId).getPath());
+        properties.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, 
Serdes.String().getClass());
+        properties.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, 
Serdes.Long().getClass());
+        properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
+        properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
+        properties.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, 
StreamsConfig.OPTIMIZE);
+        return properties;
+    }
+
+    @After
+    public void shutdown() throws InterruptedException {
+        if (kafkaStreams != null) {
+            kafkaStreams.close(Duration.ofSeconds(30));
+        }
+        if (streamsNamedTopologyWrapper != null) {
+            streamsNamedTopologyWrapper.close(Duration.ofSeconds(30));
+        }
+        CLUSTER.deleteTopicsAndWait(INPUT_STREAM_1);
+        CLUSTER.deleteTopicsAndWait(INPUT_STREAM_2);
+        CLUSTER.deleteTopicsAndWait(OUTPUT_STREAM_1);
+        CLUSTER.deleteTopicsAndWait(OUTPUT_STREAM_2);
+    }
+
+    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 {
+        // Create KafkaStreams instance
+        final StreamsBuilder builder = new StreamsBuilder();
+        
builder.stream(INPUT_STREAM_1).groupByKey().count().toStream().to(OUTPUT_STREAM_1);
+
+        kafkaStreams = new KafkaStreams(builder.build(props()), props());
+        kafkaStreams.start();
+        waitForApplicationState(singletonList(kafkaStreams), State.RUNNING, 
STARTUP_TIMEOUT);
+
+        // Write data
+        produceToInputTopics(INPUT_STREAM_1, STANDARD_INPUT_DATA);
+
+        // Verify output
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, 
OUTPUT_STREAM_1, 3),
+            CoreMatchers.equalTo(COUNT_OUTPUT_DATA));
+
+        // Pause KafkaStreams/topology
+        kafkaStreams.pause();
+        assertTrue(kafkaStreams.isPaused());
+
+        // Write more data
+        produceToInputTopics(INPUT_STREAM_1, STANDARD_INPUT_DATA);
+
+        // Verify that consumers read new data -- AKA, there is no lag.
+        final Map<String, Map<Integer, LagInfo>> lagMap = 
kafkaStreams.allLocalStorePartitionLags();
+        assertNoLag(lagMap);
+
+        // Verify no output somehow?
+        // Is there a better way to show this?
+        assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, 
OUTPUT_STREAM_1, 0),
+            CoreMatchers.equalTo(Collections.emptyList()));

Review Comment:
   As you write in the comments, this is not straightforward. I think it is not 
good to just wait for no outputs for two reasons:
   - it increases the duration of the test unnecessarily
   - it opens the door for flakiness because we only rely on a duration for 
verification
   
   I have the following idea. You create two identical `KafkaStreams` clients 
(be careful to specify distinct state directories for each). Both clients read 
from the same input topics but write to distinct output topics. You do the 
initial setup verifications for both. Then you pause one of the clients and 
wait until the other client produced a given amount of records to the output 
topic. When that happened, you verify if the paused client has not written 
anything to its output topic. The assumption is that both clients produce with 
the same rate to their output topics if they are not paused. If one of the two 
is paused and does not produce any records to the output topic in the same time 
the other client produces a certain number of records we can assume the pausing 
works.



##########
streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java:
##########
@@ -1761,6 +1761,96 @@ public void shouldUpdateStandbyTask() throws Exception {
         thread.taskManager().shutdown(true);
     }
 
+    @SuppressWarnings("unchecked")
+    @Test
+    public void shouldNotUpdateStandbyTaskWhenPaused() throws Exception {
+        final String storeName1 = "count-one";
+        final String storeName2 = "table-two";
+        final String changelogName1 = APPLICATION_ID + "-" + storeName1 + 
"-changelog";
+        final String changelogName2 = APPLICATION_ID + "-" + storeName2 + 
"-changelog";
+        final TopicPartition partition1 = new TopicPartition(changelogName1, 
1);
+        final TopicPartition partition2 = new TopicPartition(changelogName2, 
1);
+        internalStreamsBuilder
+            .stream(Collections.singleton(topic1), consumed)
+            .groupByKey()
+            .count(Materialized.as(storeName1));
+        final MaterializedInternal<Object, Object, KeyValueStore<Bytes, 
byte[]>> materialized
+            = new MaterializedInternal<>(Materialized.as(storeName2), 
internalStreamsBuilder, "");
+        internalStreamsBuilder.table(topic2, new ConsumedInternal<>(), 
materialized);
+
+        internalStreamsBuilder.buildAndOptimizeTopology();
+        final StreamThread thread = createStreamThread(CLIENT_ID, config, 
false);
+        final MockConsumer<byte[], byte[]> restoreConsumer = 
clientSupplier.restoreConsumer;
+        restoreConsumer.updatePartitions(changelogName1,
+            Collections.singletonList(new PartitionInfo(changelogName1, 1, 
null, new Node[0], new Node[0]))
+        );
+
+        restoreConsumer.updateEndOffsets(Collections.singletonMap(partition1, 
10L));
+        
restoreConsumer.updateBeginningOffsets(Collections.singletonMap(partition1, 
0L));
+        restoreConsumer.updateEndOffsets(Collections.singletonMap(partition2, 
10L));
+        
restoreConsumer.updateBeginningOffsets(Collections.singletonMap(partition2, 
0L));
+        final OffsetCheckpoint checkpoint
+            = new OffsetCheckpoint(new 
File(stateDirectory.getOrCreateDirectoryForTask(task3), CHECKPOINT_FILE_NAME));
+        checkpoint.write(Collections.singletonMap(partition2, 5L));
+
+        thread.setState(StreamThread.State.STARTING);
+        thread.rebalanceListener().onPartitionsRevoked(Collections.emptySet());
+
+        final Map<TaskId, Set<TopicPartition>> standbyTasks = new HashMap<>();
+
+        // assign single partition
+        standbyTasks.put(task1, Collections.singleton(t1p1));
+        standbyTasks.put(task3, Collections.singleton(t2p1));
+
+        thread.taskManager().handleAssignment(emptyMap(), standbyTasks);
+        thread.taskManager().tryToCompleteRestoration(mockTime.milliseconds(), 
null);
+
+        
thread.rebalanceListener().onPartitionsAssigned(Collections.emptyList());
+
+        thread.runOnce();
+
+        final StandbyTask standbyTask1 = standbyTask(thread.taskManager(), 
t1p1);
+        final StandbyTask standbyTask2 = standbyTask(thread.taskManager(), 
t2p1);
+        assertEquals(task1, standbyTask1.id());
+        assertEquals(task3, standbyTask2.id());
+
+        final KeyValueStore<Object, Long> store1 = (KeyValueStore<Object, 
Long>) standbyTask1.getStore(storeName1);
+        final KeyValueStore<Object, Long> store2 = (KeyValueStore<Object, 
Long>) standbyTask2.getStore(storeName2);
+        assertEquals(0L, store1.approximateNumEntries());
+        assertEquals(0L, store2.approximateNumEntries());
+
+        // let the store1 be restored from 0 to 10; store2 be restored from 5 
(checkpointed) to 10
+        for (long i = 0L; i < 10L; i++) {
+            restoreConsumer.addRecord(new ConsumerRecord<>(
+                changelogName1,
+                1,
+                i,
+                ("K" + i).getBytes(),
+                ("V" + i).getBytes()));
+            restoreConsumer.addRecord(new ConsumerRecord<>(
+                changelogName2,
+                1,
+                i,
+                ("K" + i).getBytes(),
+                ("V" + i).getBytes()));
+        }
+
+        // Simulate pause
+        
thread.taskManager().topologyMetadata().pauseTopology(TopologyMetadata.UNNAMED_TOPOLOGY);
+        thread.runOnce();
+        assertEquals(0L, store1.approximateNumEntries());
+        assertEquals(0L, store2.approximateNumEntries());
+
+        // Simulate resume
+        
thread.taskManager().topologyMetadata().resumeTopology(TopologyMetadata.UNNAMED_TOPOLOGY);
+        thread.runOnce();
+
+        assertEquals(10L, store1.approximateNumEntries());
+        assertEquals(4L, store2.approximateNumEntries());

Review Comment:
   Ah, I see! Could you please extract common code (especially the setup code) 
of those two tests into a method.
   We could consider to just check for greater 0.
   Additionally to standbys, we should also test if active tasks in restoration 
are paused. 



##########
streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskExecutionMetadataTest.java:
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.processor.internals;
+
+import static 
org.apache.kafka.streams.processor.internals.TopologyMetadata.UNNAMED_TOPOLOGY;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.kafka.streams.processor.TaskId;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TaskExecutionMetadataTest {
+    final static String TOPOLOGY1 = "topology1";
+    final static String TOPOLOGY2 = "topology2";
+    final static Set<String> NAMED_TOPOLOGIES = new 
HashSet<>(Arrays.asList(TOPOLOGY1, TOPOLOGY2));
+
+    @Test
+    public void testCanProcessWithoutNamedTopologies() {
+        final Set<String> topologies = Collections.singleton(UNNAMED_TOPOLOGY);
+        final Set<String> pausedTopologies = ConcurrentHashMap.newKeySet();
+
+        final TaskExecutionMetadata metadata = new 
TaskExecutionMetadata(topologies,
+            pausedTopologies);
+
+        final Task mockTask = createMockTask(UNNAMED_TOPOLOGY);
+
+        Assert.assertTrue(metadata.canProcessTask(mockTask, 0));
+        // This pauses an UNNAMED_TOPOLOGY / a KafkaStreams instance without 
named/modular
+        // topologies.
+        pausedTopologies.add(UNNAMED_TOPOLOGY);
+        Assert.assertFalse(metadata.canProcessTask(mockTask, 0));
+    }
+
+    @Test
+    public void testNamedTopologiesCanBePausedIndependently() {
+        final Set<String> pausedTopologies = ConcurrentHashMap.newKeySet();
+        final TaskExecutionMetadata metadata = new 
TaskExecutionMetadata(NAMED_TOPOLOGIES,
+            pausedTopologies);
+
+        final Task mockTask1 = createMockTask(TOPOLOGY1);
+        final Task mockTask2 = createMockTask(TOPOLOGY2);
+
+        Assert.assertTrue(metadata.canProcessTask(mockTask1, 0));
+        Assert.assertTrue(metadata.canProcessTask(mockTask2, 0));
+
+        pausedTopologies.add(TOPOLOGY1);
+        Assert.assertFalse(metadata.canProcessTask(mockTask1, 0));
+        Assert.assertTrue(metadata.canProcessTask(mockTask2, 0));
+
+        pausedTopologies.remove(TOPOLOGY1);
+        Assert.assertTrue(metadata.canProcessTask(mockTask1, 0));
+        Assert.assertTrue(metadata.canProcessTask(mockTask2, 0));
+    }
+
+    @Test
+    public void testNamedTopologiesCanBeStartedPaused() {
+        final Set<String> pausedTopologies = ConcurrentHashMap.newKeySet();
+        pausedTopologies.add(TOPOLOGY1);
+
+        final TaskExecutionMetadata metadata = new 
TaskExecutionMetadata(NAMED_TOPOLOGIES,
+            pausedTopologies);
+
+        final Task mockTask1 = createMockTask(TOPOLOGY1);
+        final Task mockTask2 = createMockTask(TOPOLOGY2);
+
+        Assert.assertFalse(metadata.canProcessTask(mockTask1, 0));
+        Assert.assertTrue(metadata.canProcessTask(mockTask2, 0));
+
+        pausedTopologies.remove(TOPOLOGY1);
+        Assert.assertTrue(metadata.canProcessTask(mockTask1, 0));
+        Assert.assertTrue(metadata.canProcessTask(mockTask2, 0));
+    }
+
+    @Test
+    public void testNamedTopologiesCanBackoff() {
+        final Set<String> pausedTopologies = ConcurrentHashMap.newKeySet();
+
+        final TaskExecutionMetadata metadata = new 
TaskExecutionMetadata(NAMED_TOPOLOGIES,
+            pausedTopologies);
+
+        final Task mockTask1 = createMockTask(TOPOLOGY1);
+        final Task mockTask2 = createMockTask(TOPOLOGY2);
+
+        Assert.assertTrue(metadata.canProcessTask(mockTask1, 0));
+        Assert.assertTrue(metadata.canProcessTask(mockTask2, 0));
+
+        metadata.registerTaskError(mockTask1, new Throwable("Error"), 0);
+        // One second after the error, task1 cannot process, task2 can.
+        Assert.assertFalse(metadata.canProcessTask(mockTask1, 1000));
+        Assert.assertTrue(metadata.canProcessTask(mockTask2, 1000));
+

Review Comment:
   Could you please use `CONSTANT_BACKOFF_MS - 1` for the verifications on line 
111 and 112? I think it makes the intent clearer.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

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

Reply via email to