lianetm commented on code in PR #17823: URL: https://github.com/apache/kafka/pull/17823#discussion_r1844247505
########## 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 + ) { + // If there is already a task in-flight, we reject adding a new one. + if (tasks.containsKey(key)) return false; + + // We use the task as a lock for the key. + tasks.put(key, task); + + // Submit the task. + executor.submit(() -> { + // If the task associated with the task is not us, it means + // that the task was either replaced or cancelled. We stop. + if (tasks.get(key) != task) return; + + // Executor the task. + final TaskResult<R> result = executeTask(task); + + // Schedule the operation. + runtime.scheduleWriteOperation( + key, + shard, + Duration.ofMillis(Long.MAX_VALUE), + coordinator -> { + // If the task associated with the task is not us, it means + // that the task was either replaced or cancelled. We stop. + if (!tasks.remove(key, task)) { + return new CoordinatorResult<>(Collections.emptyList(), null); + } + + // Call the underlying write operation with the result of the task. + return operation.onComplete(result.result, result.exception); + } + ).exceptionally(exception -> { + if (exception instanceof NotCoordinatorException || exception instanceof CoordinatorLoadInProgressException) { Review Comment: could we end up here due to a failure on scheduling the write op, without having removed the task? (ln 102). Wondering if we need to remove it here too to clean up. ########## 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 + ) { + // If there is already a task in-flight, we reject adding a new one. + if (tasks.containsKey(key)) return false; + + // We use the task as a lock for the key. + tasks.put(key, task); Review Comment: what about ```suggestion if (tasks.putIfAbsent(key, task) != null) return false; ``` ########## coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorExecutor.java: ########## @@ -0,0 +1,84 @@ +/* + * 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.KafkaException; + +/** + * An interface to schedule and cancel asynchronous tasks. The TaskRunnable + * interface defines the tasks to be executed in the executor and the + * TaskOperation defines the operation scheduled to the runtime to + * process the output of the executed task. + * + * @param <T> The record type. + */ +public interface CoordinatorExecutor<T> { + /** + * The task's runnable. + * + * @param <R> The return type. + */ + interface TaskRunnable<R> { + R run() throws Throwable; + } + + /** + * The task's write operation to handle the output + * of the task. + * + * @param <T> The record type. + * @param <R> The return type of the task. + */ + interface TaskOperation<T, R> { + CoordinatorResult<Void, T> onComplete( + R result, + Throwable exception + ) throws KafkaException; + } + + /** + * Schedule an asynchronous tasks. Note that only one task for a given key can Review Comment: ```suggestion * Schedule an asynchronous task. Note that only one task for a given key can ``` ########## 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 + ) { + // If there is already a task in-flight, we reject adding a new one. + if (tasks.containsKey(key)) return false; + + // We use the task as a lock for the key. + tasks.put(key, task); + + // Submit the task. + executor.submit(() -> { + // If the task associated with the task is not us, it means + // that the task was either replaced or cancelled. We stop. + if (tasks.get(key) != task) return; + + // Executor the task. Review Comment: ```suggestion // Execute the task. ``` -- 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]
