guozhangwang commented on a change in pull request #8988:
URL: https://github.com/apache/kafka/pull/8988#discussion_r506650596



##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/StateRestoreThread.java
##########
@@ -0,0 +1,316 @@
+/*
+ * 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 org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.InterruptException;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.errors.StreamsException;
+import org.apache.kafka.streams.errors.TaskCorruptedException;
+import org.apache.kafka.streams.processor.StateRestoreListener;
+import org.apache.kafka.streams.processor.TaskId;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+
+/**
+ * This is the thread responsible for restoring state stores for both active 
and standby tasks
+ */
+public class StateRestoreThread extends Thread {
+
+    private final Time time;
+    private final Logger log;
+    private final ChangelogReader changelogReader;
+    private final AtomicBoolean isRunning = new AtomicBoolean(true);
+    private final CountDownLatch shutdownLatch = new CountDownLatch(1);
+    private final LinkedBlockingDeque<TaskItem> taskItemQueue;
+    private final AtomicReference<Set<TopicPartition>> completedChangelogs;
+    private final LinkedBlockingDeque<TaskCorruptedException> 
corruptedExceptions;
+    private final AtomicReference<RuntimeException> fatalException;
+
+    public boolean isRunning() {
+        return isRunning.get();
+    }
+
+    public StateRestoreThread(final Time time,
+                              final StreamsConfig config,
+                              final String threadClientId,
+                              final Admin adminClient,
+                              final String groupId,
+                              final Consumer<byte[], byte[]> restoreConsumer,
+                              final StateRestoreListener 
userStateRestoreListener) {
+        this(time, threadClientId, new StoreChangelogReader(time, config, 
threadClientId,
+                adminClient, groupId, restoreConsumer, 
userStateRestoreListener));
+    }
+
+    // for testing only
+    public StateRestoreThread(final Time time,
+                              final String threadClientId,
+                              final ChangelogReader changelogReader) {
+        super(threadClientId);
+
+        final String logPrefix = String.format("state-restore-thread [%s] ", 
threadClientId);
+        final LogContext logContext = new LogContext(logPrefix);
+
+        this.time = time;
+        this.log = logContext.logger(getClass());
+        this.taskItemQueue = new LinkedBlockingDeque<>();
+        this.fatalException = new AtomicReference<>();
+        this.corruptedExceptions = new LinkedBlockingDeque<>();
+        this.completedChangelogs = new 
AtomicReference<>(Collections.emptySet());
+
+        this.changelogReader = changelogReader;
+    }
+
+    private synchronized void waitIfAllChangelogsCompleted() {
+        final Set<TopicPartition> allChangelogs = 
changelogReader.allChangelogs();
+        if (allChangelogs.equals(changelogReader.completedChangelogs())) {
+            log.debug("All changelogs {} have completed restoration so far, 
will wait " +
+                    "until new changelogs are registered", allChangelogs);
+
+            while (isRunning.get() && taskItemQueue.isEmpty()) {
+                try {
+                    wait();
+                } catch (final InterruptedException e) {
+                    // do nothing
+                }
+            }
+        }
+    }
+
+    public synchronized void addInitializedTasks(final List<Task> tasks) {
+        if (!tasks.isEmpty()) {
+            for (final Task task: tasks) {
+                taskItemQueue.add(new TaskItem(task, ItemType.CREATE, 
task.changelogPartitions()));
+            }
+            notifyAll();
+        }
+    }
+
+    public synchronized void addClosedTasks(final Map<Task, 
Collection<TopicPartition>> tasks) {
+        if (!tasks.isEmpty()) {
+            for (final Map.Entry<Task, Collection<TopicPartition>> entry : 
tasks.entrySet()) {
+                taskItemQueue.add(new TaskItem(entry.getKey(), ItemType.CLOSE, 
entry.getValue()));
+            }
+            notifyAll();
+        }
+    }
+
+    public Set<TopicPartition> completedChangelogs() {
+        return completedChangelogs.get();
+    }
+
+    @Override
+    public void run() {
+        try {
+            while (isRunning()) {
+                runOnce();
+            }
+        } catch (final RuntimeException e) {
+            log.error("Encountered the following exception while restoring 
states " +
+                    "and the thread is going to shut down: ", e);
+
+            // we would not throw the exception from the restore thread
+            // but would need the main thread to get and throw it
+            fatalException.set(e);
+        } finally {
+            // if the thread is exiting due to exception,
+            // we would still set its running flag
+            isRunning.set(false);
+
+            try {
+                changelogReader.clear();
+            } catch (final Throwable e) {
+                log.error("Failed to close changelog reader due to the 
following error:", e);
+            }
+
+            shutdownLatch.countDown();
+        }
+    }
+
+    // Visible for testing
+    void runOnce() {
+        waitIfAllChangelogsCompleted();
+
+        if (!isRunning.get())
+            return;
+
+        // a task being recycled maybe in both closed and initialized tasks,
+        // and hence we should process the closed ones first and then 
initialized ones

Review comment:
       Updated the comment a bit, hopefully it would be more clear now.




----------------------------------------------------------------
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.

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


Reply via email to