-
Notifications
You must be signed in to change notification settings - Fork 14k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
KAFKA-17593; [7/N] Introduce CoordinatorExecutor #17823
base: trunk
Are you sure you want to change the base?
Changes from all commits
ab8059a
c86fa14
eb6cbe6
0808c8b
81a4ddc
c67751c
b70d1f1
a828b4a
3140776
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
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 tasks. 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); | ||
} |
Original file line number | Diff line number | Diff line change | ||||||||||
---|---|---|---|---|---|---|---|---|---|---|---|---|
@@ -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); | ||||||||||||
Comment on lines
+80
to
+83
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. what about
Suggested change
|
||||||||||||
|
||||||||||||
// 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. | ||||||||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
|
||||||||||||
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) { | ||||||||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 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. |
||||||||||||
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(); | ||||||||||||
} | ||||||||||||
} | ||||||||||||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.