forked from apache/kafka
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
KAFKA-17593; [7/N] Introduce CoordinatorExecutor (apache#17823)
This patch introduces the `CoordinatorExecutor` construct into the `CoordinatorRuntime`. It allows scheduling asynchronous tasks from within a `CoordinatorShard` while respecting the runtime semantic. It will be used to asynchronously resolve regular expressions. The `GroupCoordinatorService` uses a default `ExecutorService` with a single thread to back it at the moment. It seems that it should be sufficient. In the future, we could consider making the number of threads configurable. Reviewers: Jeff Kim <jeff.kim@confluent.io>, Lianet Magrans <lmagrans@confluent.io>
- Loading branch information
Showing
10 changed files
with
801 additions
and
4 deletions.
There are no files selected for viewing
84 changes: 84 additions & 0 deletions
84
...common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorExecutor.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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 task. Note that only one task for a given key can | ||
* be executed at the time. | ||
* | ||
* @param key The key to identify the task. | ||
* @param task The task itself. | ||
* @param operation The runtime operation to handle the output of the task. | ||
* @return True if the task was scheduled; False otherwise. | ||
* | ||
* @param <R> The return type of the task. | ||
*/ | ||
<R> boolean schedule( | ||
String key, | ||
TaskRunnable<R> task, | ||
TaskOperation<T, R> operation | ||
); | ||
|
||
/** | ||
* Return true if the key is associated to a task; false otherwise. | ||
* | ||
* @param key The key to identify the task. | ||
* @return A boolean indicating whether the task is scheduled or not. | ||
*/ | ||
boolean isScheduled(String key); | ||
|
||
/** | ||
* Cancel the given task | ||
* | ||
* @param key The key to identify the task. | ||
*/ | ||
void cancel(String key); | ||
} |
147 changes: 147 additions & 0 deletions
147
...on/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorExecutorImpl.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,147 @@ | ||
/* | ||
* 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.Iterator; | ||
import java.util.Map; | ||
import java.util.concurrent.ConcurrentHashMap; | ||
import java.util.concurrent.ExecutorService; | ||
import java.util.concurrent.RejectedExecutionException; | ||
|
||
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 Duration writeTimeout; | ||
private final Map<String, TaskRunnable<?>> tasks = new ConcurrentHashMap<>(); | ||
|
||
public CoordinatorExecutorImpl( | ||
LogContext logContext, | ||
TopicPartition shard, | ||
CoordinatorRuntime<S, U> runtime, | ||
ExecutorService executor, | ||
Duration writeTimeout | ||
) { | ||
this.log = logContext.logger(CoordinatorExecutorImpl.class); | ||
this.shard = shard; | ||
this.runtime = runtime; | ||
this.executor = executor; | ||
this.writeTimeout = writeTimeout; | ||
} | ||
|
||
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; | ||
|
||
// Submit the task. | ||
executor.submit(() -> { | ||
// If the task associated with the key is not us, it means | ||
// that the task was either replaced or cancelled. We stop. | ||
if (tasks.get(key) != task) return; | ||
|
||
// Execute the task. | ||
final TaskResult<R> result = executeTask(task); | ||
|
||
// Schedule the operation. | ||
runtime.scheduleWriteOperation( | ||
key, | ||
shard, | ||
writeTimeout, | ||
coordinator -> { | ||
// If the task associated with the key is not us, it means | ||
// that the task was either replaced or cancelled. We stop. | ||
if (!tasks.remove(key, task)) { | ||
throw new RejectedExecutionException(String.format("Task %s was overridden or cancelled", key)); | ||
} | ||
|
||
// Call the underlying write operation with the result of the task. | ||
return operation.onComplete(result.result, result.exception); | ||
} | ||
).exceptionally(exception -> { | ||
// Remove the task after a failure. | ||
tasks.remove(key, task); | ||
|
||
if (exception instanceof RejectedExecutionException) { | ||
log.debug("The write event for the task {} was not executed because it was " + | ||
"cancelled or overridden.", key); | ||
} else if (exception instanceof NotCoordinatorException || exception instanceof CoordinatorLoadInProgressException) { | ||
log.debug("The write event for the task {} failed due to {}. Ignoring it because " + | ||
"the coordinator is not active.", key, exception.getMessage()); | ||
} else { | ||
log.error("The write event for the task {} failed due to {}. Ignoring it. ", | ||
key, exception.getMessage()); | ||
} | ||
|
||
return null; | ||
}); | ||
}); | ||
|
||
return true; | ||
} | ||
|
||
@Override | ||
public boolean isScheduled(String key) { | ||
return tasks.containsKey(key); | ||
} | ||
|
||
@Override | ||
public void cancel(String key) { | ||
tasks.remove(key); | ||
} | ||
|
||
public void cancelAll() { | ||
Iterator<String> iterator = tasks.keySet().iterator(); | ||
while (iterator.hasNext()) { | ||
iterator.remove(); | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.