ableegoldman commented on code in PR #12737:
URL: https://github.com/apache/kafka/pull/12737#discussion_r993948622


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

Review Comment:
   Maybe clarify the behavior a bit further in the docs here (unless it's still 
undecided) -- eg do we expect that this interrupts the current processing of 
this task ASAP? Does the TaskExecutor do anything (like flushing) or just 
return that task when it can? etc



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

Review Comment:
   Just wondering, why does `add` work on multiple tasks but `remove` only 
removes one at a time?



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

Review Comment:
   Just wondering, who is going to be invoking this? On a related note, is the 
`TaskManager ` effectively owned by the polling thread and used to interface 
between the processing threads, or does most of the task management work happen 
on the actual processing threads?



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

Review Comment:
   Is this implying that the caller should flush the returned task itself, or 
explaining that it doesn't immediately return the task because it has to be 
flushed first?



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

Review Comment:
   Also, will this method fail if all the tasks are locked? Or does it block 
until one is ready?



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

Review Comment:
   Also, should this interface have an `#assign` method as well? I know it will 
probably make more sense as I see the next PR(s) but from the API alone, the 
asymmetry feels quite odd. Makes it confusing who is responsible for the  
task's ownership. 
   
   For example, the `TaskManager` class had both an assign and unassign type of 
API, so why does the `TaskExecutor` only have the unassign?



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