dajac commented on code in PR #17823:
URL: https://github.com/apache/kafka/pull/17823#discussion_r1847849903


##########
coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorExecutorImpl.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.coordinator.common.runtime;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.utils.LogContext;
+
+import org.slf4j.Logger;
+
+import java.time.Duration;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+
+public class CoordinatorExecutorImpl<S extends CoordinatorShard<U>, U> 
implements CoordinatorExecutor<U> {
+    private static class TaskResult<R> {
+        final R result;
+        final Throwable exception;
+
+        TaskResult(
+            R result,
+            Throwable exception
+        ) {
+            this.result = result;
+            this.exception = exception;
+        }
+    }
+
+    private final Logger log;
+    private final TopicPartition shard;
+    private final CoordinatorRuntime<S, U> runtime;
+    private final ExecutorService executor;
+    private final Map<String, TaskRunnable<?>> tasks = new 
ConcurrentHashMap<>();
+
+    public CoordinatorExecutorImpl(
+        LogContext logContext,
+        TopicPartition shard,
+        CoordinatorRuntime<S, U> runtime,
+        ExecutorService executor
+    ) {
+        this.log = logContext.logger(CoordinatorExecutorImpl.class);
+        this.shard = shard;
+        this.runtime = runtime;
+        this.executor = executor;
+    }
+
+    private <R> TaskResult<R> executeTask(TaskRunnable<R> task) {
+        try {
+            return new TaskResult<>(task.run(), null);
+        } catch (Throwable ex) {
+            return new TaskResult<>(null, ex);
+        }
+    }
+
+    @Override
+    public <R> boolean schedule(
+        String key,
+        TaskRunnable<R> task,
+        TaskOperation<U, R> operation
+    ) {
+        // Put the task if the key is free. Otherwise, reject it.
+        if (tasks.putIfAbsent(key, task) != null) return false;

Review Comment:
   Yeah. We basically want to have one asynchronous task at the time. My goal 
was to avoid spamming the queue with the same order multiple times. It will get 
clearer with the next PR.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to