This is an automated email from the ASF dual-hosted git repository.

guozhang pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 69059b5f288 Kafka Streams Threading P1: Add Interface for new 
TaskManager and TaskExecutor (#12737)
69059b5f288 is described below

commit 69059b5f288ef5b6496953ba1d77b2570d8b3142
Author: Guozhang Wang <wangg...@gmail.com>
AuthorDate: Wed Oct 12 16:33:13 2022 -0700

    Kafka Streams Threading P1: Add Interface for new TaskManager and 
TaskExecutor (#12737)
    
    The interfaces (and their future impls) are added under the 
processor/internals/tasks package, to distinguish with the existing old classes:
    
    1. TaskExecutor is the interface for a processor thread. It takes at most 
one task to process at a given time from the task manager. When being asked 
from the task manager to un-assign the current processing task, it will stop 
processing and give the task back to task manager.
    2. TaskManager schedules all the active tasks to assign to TaskExecutors. 
Specifically: 1) when a task executor ask it for an unassigned task to process 
(assignNextTask), it will return the available task based on its scheduling 
algorithm. 2) when the task manager decides to commit (all) tasks, or when a 
rebalance event requires it to modify the maintained active tasks (via 
onAssignment), it will lock all the tasks that are going to be closed / 
committed, asking the TaskExecutor to gi [...]
    
    Reviewers: John Roesler <vvcep...@apache.org>, Anna Sophie Blee-Goldman 
<ableegold...@apache.org>
---
 .../processor/internals/tasks/TaskExecutor.java    |  57 ++++++++++++
 .../processor/internals/tasks/TaskManager.java     | 100 +++++++++++++++++++++
 2 files changed, 157 insertions(+)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/tasks/TaskExecutor.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/tasks/TaskExecutor.java
new file mode 100644
index 00000000000..ead1fb8179e
--- /dev/null
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/tasks/TaskExecutor.java
@@ -0,0 +1,57 @@
+/*
+ * 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.tasks;
+
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.streams.processor.internals.ReadOnlyTask;
+import org.apache.kafka.streams.processor.internals.StreamTask;
+
+import java.time.Duration;
+
+public interface TaskExecutor {
+
+    /**
+     * Starts the task processor.
+     */
+    void start();
+
+    /**
+     * Shuts down the task processor updater.
+     *
+     * @param timeout duration how long to wait until the state updater is 
shut down
+     *
+     * @throws
+     *     org.apache.kafka.streams.errors.StreamsException if the state 
updater thread cannot shutdown within the timeout
+     */
+    void shutdown(final Duration timeout);
+
+    /**
+     * Get the current assigned processing task. The task returned is 
read-only and cannot be modified.
+     *
+     * @return the current processing task
+     */
+    ReadOnlyTask currentTask();
+
+    /**
+     * Unassign the current processing task from the task processor and give 
it back to the state manager.
+     *
+     * The paused task must be flushed since it may be committed or closed by 
the task manager next.
+     *
+     * This method does not block, instead a future is returned.
+     */
+    KafkaFuture<StreamTask> unassign();
+}
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/tasks/TaskManager.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/tasks/TaskManager.java
new file mode 100644
index 00000000000..e9929714aca
--- /dev/null
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/tasks/TaskManager.java
@@ -0,0 +1,100 @@
+/*
+ * 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.tasks;
+
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.streams.processor.TaskId;
+import org.apache.kafka.streams.processor.internals.ReadOnlyTask;
+import org.apache.kafka.streams.processor.internals.StreamTask;
+import java.util.Set;
+
+public interface TaskManager {
+
+    /**
+     * Get the next processible active task for the requested executor. Once 
the task is assigned to
+     * the requested task executor, it should not be assigned to any other 
executors until it was
+     * returned to the task manager.
+     *
+     * @param executor the requesting {@link TaskExecutor}
+     */
+    StreamTask assignNextTask(final TaskExecutor executor);
+
+    /**
+     * Unassign the stream task so that it can be assigned to other executors 
later
+     * or be removed from the task manager. The requested executor must have 
locked
+     * the task already, otherwise an exception would be thrown.
+     *
+     * @param executor the requesting {@link TaskExecutor}
+     */
+    void unassignTask(final StreamTask task, final TaskExecutor executor);
+
+    /**
+     * Lock a set of active tasks from the task manager so that they will not 
be assigned to
+     * any {@link TaskExecutor}s anymore until they are unlocked. At the time 
this function
+     * is called, the requested tasks may already be locked by some {@link 
TaskExecutor}s,
+     * and in that case the task manager need to first unassign these tasks 
from the
+     * executors.
+     *
+     * This function is needed when we need to 1) commit these tasks, 2) 
remove these tasks.
+     *
+     * This method does not block, instead a future is returned.
+     */
+    KafkaFuture<Void> lockTasks(final Set<TaskId> taskIds);
+
+    /**
+     * Lock all of the managed active tasks from the task manager. Similar to 
{@link #lockTasks(Set)}.
+     *
+     * This method does not block, instead a future is returned.
+     */
+    KafkaFuture<Void> lockAllTasks();
+
+    /**
+     * Unlock the tasks so that they can be assigned to executors
+     */
+    void unlockTasks(final Set<TaskId> taskIds);
+
+    /**
+     * Unlock all of the managed active tasks from the task manager. Similar 
to {@link #unlockTasks(Set)}.
+     *
+     * This method does not block, instead a future is returned.
+     */
+    void unlockAllTasks();
+
+    /**
+     * Add a new active task to the task manager.
+     *
+     * @param tasks task to add
+     */
+    void add(final Set<StreamTask> tasks);
+
+    /**
+     * Remove an active task from the task manager.
+     *
+     * The task to remove must be locked.
+     *
+     * @param taskId ID of the task to remove
+     */
+    void remove(final TaskId taskId);
+
+    /**
+     * Gets all active tasks that are managed by this manager. The returned 
tasks are read-only
+     * and cannot be manipulated.
+     *
+     * @return set of all managed active tasks
+     */
+    Set<ReadOnlyTask> getTasks();
+}

Reply via email to