[ 
https://issues.apache.org/jira/browse/KAFKA-6844?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16464450#comment-16464450
 ] 

ASF GitHub Bot commented on KAFKA-6844:
---------------------------------------

guozhangwang closed pull request #4950: KAFKA-6844: Call shutdown on 
GlobalStreamThread after all StreamThreads have stopped
URL: https://github.com/apache/kafka/pull/4950
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java 
b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
index eed12f15b4f..56d031b746e 100644
--- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
+++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
@@ -876,10 +876,6 @@ public void run() {
                         thread.setStateListener(null);
                         thread.shutdown();
                     }
-                    if (globalStreamThread != null) {
-                        globalStreamThread.setStateListener(null);
-                        globalStreamThread.shutdown();
-                    }
 
                     for (final StreamThread thread : threads) {
                         try {
@@ -890,6 +886,12 @@ public void run() {
                             Thread.currentThread().interrupt();
                         }
                     }
+
+                    if (globalStreamThread != null) {
+                        globalStreamThread.setStateListener(null);
+                        globalStreamThread.shutdown();
+                    }
+
                     if (globalStreamThread != null && 
!globalStreamThread.stillRunning()) {
                         try {
                             globalStreamThread.join();
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java
new file mode 100644
index 00000000000..c7b63ad0b61
--- /dev/null
+++ 
b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java
@@ -0,0 +1,217 @@
+/*
+ * 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 org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.common.serialization.LongSerializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.streams.Consumed;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.KeyValue;
+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.KStream;
+import org.apache.kafka.streams.processor.AbstractProcessor;
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.ProcessorSupplier;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.kafka.streams.state.Stores;
+import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder;
+import org.apache.kafka.test.IntegrationTest;
+import org.apache.kafka.test.MockProcessorSupplier;
+import org.apache.kafka.test.TestCondition;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+import kafka.utils.MockTime;
+
+import static org.junit.Assert.assertEquals;
+
+@Category({IntegrationTest.class})
+public class GlobalThreadShutDownOrderTest {
+
+    private static final int NUM_BROKERS = 1;
+    private static final Properties BROKER_CONFIG;
+
+    static {
+        BROKER_CONFIG = new Properties();
+        BROKER_CONFIG.put("transaction.state.log.replication.factor", (short) 
1);
+        BROKER_CONFIG.put("transaction.state.log.min.isr", 1);
+    }
+
+    @ClassRule
+    public static final EmbeddedKafkaCluster CLUSTER =
+        new EmbeddedKafkaCluster(NUM_BROKERS, BROKER_CONFIG);
+
+    private final MockTime mockTime = CLUSTER.time;
+    private final String globalStore = "globalStore";
+    private StreamsBuilder builder;
+    private Properties streamsConfiguration;
+    private KafkaStreams kafkaStreams;
+    private String globalStoreTopic;
+    private String streamTopic;
+    private KStream<String, Long> stream;
+    private List<Long> retrievedValuesList = new ArrayList<>();
+    private boolean firstRecordProcessed;
+
+    @Before
+    public void before() throws InterruptedException {
+
+        builder = new StreamsBuilder();
+        createTopics();
+        streamsConfiguration = new Properties();
+        final String applicationId = "global-thread-shutdown-test";
+        streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, 
applicationId);
+        streamsConfiguration
+            .put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, 
CLUSTER.bootstrapServers());
+        streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, 
"earliest");
+        streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, 
TestUtils.tempDirectory().getPath());
+        
streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
+        
streamsConfiguration.put(IntegrationTestUtils.INTERNAL_LEAVE_GROUP_ON_CLOSE, 
true);
+        streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100);
+
+        final Consumed<String, Long> stringLongConsumed = 
Consumed.with(Serdes.String(), Serdes.Long());
+
+        KeyValueStoreBuilder<String, Long> storeBuilder = new 
KeyValueStoreBuilder<>(Stores.persistentKeyValueStore(globalStore),
+                                                                               
      Serdes.String(),
+                                                                               
      Serdes.Long(),
+                                                                               
      mockTime);
+
+        builder.addGlobalStore(storeBuilder,
+                               globalStoreTopic,
+                               Consumed.with(Serdes.String(), Serdes.Long()),
+                               new MockProcessorSupplier());
+
+        stream = builder.stream(streamTopic, stringLongConsumed);
+
+        stream.process(new ProcessorSupplier<String, Long>() {
+            @Override
+            public Processor<String, Long> get() {
+                return new GlobalStoreProcessor(globalStore);
+            }
+        });
+
+    }
+
+    @After
+    public void whenShuttingDown() throws IOException {
+        if (kafkaStreams != null) {
+            kafkaStreams.close();
+        }
+        IntegrationTestUtils.purgeLocalStreamsState(streamsConfiguration);
+    }
+
+
+    @Test
+    public void shouldFinishGlobalStoreOperationOnShutDown() throws Exception {
+        kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration);
+        populateTopics(globalStoreTopic);
+        populateTopics(streamTopic);
+
+        kafkaStreams.start();
+
+        TestUtils.waitForCondition(new TestCondition() {
+            @Override
+            public boolean conditionMet() {
+                return firstRecordProcessed;
+            }
+        }, 5000L, "Has not processed record within 5 seconds");
+
+        kafkaStreams.close(30, TimeUnit.SECONDS);
+
+        List<Long> expectedRetrievedValues = Arrays.asList(1L, 2L, 3L, 4L);
+        assertEquals(expectedRetrievedValues, retrievedValuesList);
+    }
+
+
+    private void createTopics() throws InterruptedException {
+        streamTopic = "stream-topic";
+        globalStoreTopic = "global-store-topic";
+        CLUSTER.createTopics(streamTopic);
+        CLUSTER.createTopic(globalStoreTopic);
+    }
+
+
+    private void populateTopics(String topicName) throws Exception {
+        IntegrationTestUtils.produceKeyValuesSynchronously(
+            topicName,
+            Arrays.asList(
+                new KeyValue<>("A", 1L),
+                new KeyValue<>("B", 2L),
+                new KeyValue<>("C", 3L),
+                new KeyValue<>("D", 4L)),
+            TestUtils.producerConfig(
+                CLUSTER.bootstrapServers(),
+                StringSerializer.class,
+                LongSerializer.class,
+                new Properties()),
+            mockTime);
+    }
+
+
+    private class GlobalStoreProcessor extends AbstractProcessor<String, Long> 
{
+
+        private KeyValueStore<String, Long> store;
+        private final String storeName;
+
+        GlobalStoreProcessor(final String storeName) {
+
+            this.storeName = storeName;
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public void init(final ProcessorContext context) {
+            super.init(context);
+            store = (KeyValueStore<String, Long>) 
context.getStateStore(storeName);
+        }
+
+        @Override
+        public void process(final String key, final Long value) {
+            firstRecordProcessed = true;
+        }
+
+
+        @Override
+        public void close() {
+            List<String> keys = Arrays.asList("A", "B", "C", "D");
+            for (String key : keys) {
+                // need to simulate thread slow in closing
+                Utils.sleep(1000);
+                retrievedValuesList.add(store.get(key));
+            }
+        }
+    }
+
+}


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Race condition between StreamThread and GlobalStreamThread stopping
> -------------------------------------------------------------------
>
>                 Key: KAFKA-6844
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6844
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>    Affects Versions: 1.0.0, 1.1.0, 1.0.1
>            Reporter: Bill Bejeck
>            Assignee: Bill Bejeck
>            Priority: Major
>             Fix For: 2.0.0
>
>
> There can be a race condition where shut down is called on a StreamThread 
> then shut down is called on a GlobalStreamThread, but the StreamThread can be 
> delayed in shutting down,  and the GlobalStreamThread can shutdown first.
>  
> If the StreamThread tries to access a GlobalStateStore before closing the 
> user can get an exception stating  "..{{Store xxx is currently closed "}}
> Here's a redacted partial log file showing this process:
> {{2018-04-23 12:54:10 [INFO] [logger] DataExportTopology:86 - Closing 
> streams}}
>  {{2018-04-23 12:54:10 [INFO] [logger] KafkaStreams:346 - stream-client 
> [redacted-info] State transition from RUNNING to PENDING_SHUTDOWN}}
>  {{2018-04-23 12:54:10 [INFO] [logger] StreamThread:336 - stream-thread 
> [redacted-info-StreamThread-1] Informed to shut down}}
>  {{2018-04-23 12:54:10 [INFO] [logger] StreamThread:346 - stream-thread 
> [redacted-info-StreamThread-1] State transition from RUNNING to 
> PENDING_SHUTDOWN}}
>  {{2018-04-23 12:54:10 [INFO] [logger] StreamThread:336 - stream-thread 
> [redacted-info-StreamThread-2] Informed to shut down}}
>  {{2018-04-23 12:54:10 [INFO] [logger] StreamThread:346 - stream-thread 
> [redacted-info-StreamThread-2] State transition from RUNNING to 
> PENDING_SHUTDOWN}}
>  {{2018-04-23 12:54:10 [INFO] [logger] StreamThread:336 - stream-thread 
> [redacted-info-StreamThread-3] Informed to shut down}}
>  {{2018-04-23 12:54:10 [INFO] [logger] StreamThread:346 - stream-thread 
> [redacted-info-StreamThread-3] State transition from RUNNING to 
> PENDING_SHUTDOWN}}
>  {{2018-04-23 12:54:10 [INFO] [logger] StreamThread:336 - stream-thread 
> [redacted-info-StreamThread-4] Informed to shut down}}
>  {{2018-04-23 12:54:10 [INFO] [logger] StreamThread:346 - stream-thread 
> [redacted-info-StreamThread-4] State transition from RUNNING to 
> PENDING_SHUTDOWN}}
>  {{2018-04-23 12:54:10 [INFO] [logger] GlobalStreamThread:346 - 
> global-stream-thread [redacted-info-GlobalStreamThread] State transition from 
> RUNNING to PENDING_SHUTDOWN}}
>  {{2018-04-23 12:54:10 [INFO] [logger] StreamThread:336 - stream-thread 
> [redacted-info-StreamThread-4] Shutting down}}
>  {{2018-04-23 12:54:10 [INFO] [logger] StreamThread:336 - stream-thread 
> [redacted-info-StreamThread-1] Shutting down}}
>  {{2018-04-23 12:54:10 [INFO] [logger] GlobalStreamThread:336 - 
> global-stream-thread [redacted-info-GlobalStreamThread] Shutting down}}
>  {{2018-04-23 12:54:10 [INFO] [logger] GlobalStreamThread:346 - 
> global-stream-thread [redacted-info-GlobalStreamThread] State transition from 
> PENDING_SHUTDOWN to DEAD}}
>  {{2018-04-23 12:54:10 [INFO] [logger] GlobalStreamThread:336 - 
> global-stream-thread [redacted-info-GlobalStreamThread] Shutdown complete}}
>  {{2018-04-23 12:54:10 [INFO] [logger] StreamThread:336 - stream-thread 
> [redacted-info-StreamThread-2] Shutting down}}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to