mirror of https://github.com/apache/kafka.git
KAFKA-17593; [7/N] Introduce CoordinatorExecutor (#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>
This commit is contained in:
parent
a334b1b6fd
commit
a211ee99b5
|
@ -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);
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -60,6 +60,7 @@ import java.util.Optional;
|
|||
import java.util.OptionalInt;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.RejectedExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
@ -118,6 +119,7 @@ public class CoordinatorRuntime<S extends CoordinatorShard<U>, U> implements Aut
|
|||
private Serializer<U> serializer;
|
||||
private Compression compression;
|
||||
private int appendLingerMs;
|
||||
private ExecutorService executorService;
|
||||
|
||||
public Builder<S, U> withLogPrefix(String logPrefix) {
|
||||
this.logPrefix = logPrefix;
|
||||
|
@ -189,6 +191,11 @@ public class CoordinatorRuntime<S extends CoordinatorShard<U>, U> implements Aut
|
|||
return this;
|
||||
}
|
||||
|
||||
public Builder<S, U> withExecutorService(ExecutorService executorService) {
|
||||
this.executorService = executorService;
|
||||
return this;
|
||||
}
|
||||
|
||||
public CoordinatorRuntime<S, U> build() {
|
||||
if (logPrefix == null)
|
||||
logPrefix = "";
|
||||
|
@ -216,6 +223,8 @@ public class CoordinatorRuntime<S extends CoordinatorShard<U>, U> implements Aut
|
|||
compression = Compression.NONE;
|
||||
if (appendLingerMs < 0)
|
||||
throw new IllegalArgumentException("AppendLinger must be >= 0");
|
||||
if (executorService == null)
|
||||
throw new IllegalArgumentException("ExecutorService must be set.");
|
||||
|
||||
return new CoordinatorRuntime<>(
|
||||
logPrefix,
|
||||
|
@ -231,7 +240,8 @@ public class CoordinatorRuntime<S extends CoordinatorShard<U>, U> implements Aut
|
|||
coordinatorMetrics,
|
||||
serializer,
|
||||
compression,
|
||||
appendLingerMs
|
||||
appendLingerMs,
|
||||
executorService
|
||||
);
|
||||
}
|
||||
}
|
||||
|
@ -551,6 +561,11 @@ public class CoordinatorRuntime<S extends CoordinatorShard<U>, U> implements Aut
|
|||
*/
|
||||
final EventBasedCoordinatorTimer timer;
|
||||
|
||||
/**
|
||||
* The coordinator executor.
|
||||
*/
|
||||
final CoordinatorExecutorImpl<S, U> executor;
|
||||
|
||||
/**
|
||||
* The current state.
|
||||
*/
|
||||
|
@ -603,6 +618,13 @@ public class CoordinatorRuntime<S extends CoordinatorShard<U>, U> implements Aut
|
|||
this.epoch = -1;
|
||||
this.deferredEventQueue = new DeferredEventQueue(logContext);
|
||||
this.timer = new EventBasedCoordinatorTimer(tp, logContext);
|
||||
this.executor = new CoordinatorExecutorImpl<>(
|
||||
logContext,
|
||||
tp,
|
||||
CoordinatorRuntime.this,
|
||||
executorService,
|
||||
defaultWriteTimeout
|
||||
);
|
||||
this.bufferSupplier = new BufferSupplier.GrowableBufferSupplier();
|
||||
}
|
||||
|
||||
|
@ -633,6 +655,7 @@ public class CoordinatorRuntime<S extends CoordinatorShard<U>, U> implements Aut
|
|||
.withSnapshotRegistry(snapshotRegistry)
|
||||
.withTime(time)
|
||||
.withTimer(timer)
|
||||
.withExecutor(executor)
|
||||
.withCoordinatorMetrics(coordinatorMetrics)
|
||||
.withTopicPartition(tp)
|
||||
.build(),
|
||||
|
@ -714,6 +737,7 @@ public class CoordinatorRuntime<S extends CoordinatorShard<U>, U> implements Aut
|
|||
highWatermarklistener = null;
|
||||
}
|
||||
timer.cancelAll();
|
||||
executor.cancelAll();
|
||||
deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
|
||||
failCurrentBatch(Errors.NOT_COORDINATOR.exception());
|
||||
if (coordinator != null) {
|
||||
|
@ -1899,6 +1923,12 @@ public class CoordinatorRuntime<S extends CoordinatorShard<U>, U> implements Aut
|
|||
*/
|
||||
private final int appendLingerMs;
|
||||
|
||||
/**
|
||||
* The executor service used by the coordinator runtime to schedule
|
||||
* asynchronous tasks.
|
||||
*/
|
||||
private final ExecutorService executorService;
|
||||
|
||||
/**
|
||||
* Atomic boolean indicating whether the runtime is running.
|
||||
*/
|
||||
|
@ -1926,6 +1956,7 @@ public class CoordinatorRuntime<S extends CoordinatorShard<U>, U> implements Aut
|
|||
* @param serializer The serializer.
|
||||
* @param compression The compression codec.
|
||||
* @param appendLingerMs The append linger time in ms.
|
||||
* @param executorService The executor service.
|
||||
*/
|
||||
@SuppressWarnings("checkstyle:ParameterNumber")
|
||||
private CoordinatorRuntime(
|
||||
|
@ -1942,7 +1973,8 @@ public class CoordinatorRuntime<S extends CoordinatorShard<U>, U> implements Aut
|
|||
CoordinatorMetrics coordinatorMetrics,
|
||||
Serializer<U> serializer,
|
||||
Compression compression,
|
||||
int appendLingerMs
|
||||
int appendLingerMs,
|
||||
ExecutorService executorService
|
||||
) {
|
||||
this.logPrefix = logPrefix;
|
||||
this.logContext = logContext;
|
||||
|
@ -1960,6 +1992,7 @@ public class CoordinatorRuntime<S extends CoordinatorShard<U>, U> implements Aut
|
|||
this.serializer = serializer;
|
||||
this.compression = compression;
|
||||
this.appendLingerMs = appendLingerMs;
|
||||
this.executorService = executorService;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2423,6 +2456,7 @@ public class CoordinatorRuntime<S extends CoordinatorShard<U>, U> implements Aut
|
|||
}
|
||||
});
|
||||
coordinators.clear();
|
||||
executorService.shutdown();
|
||||
Utils.closeQuietly(runtimeMetrics, "runtime metrics");
|
||||
log.info("Coordinator runtime closed.");
|
||||
}
|
||||
|
|
|
@ -75,6 +75,17 @@ public interface CoordinatorShardBuilder<S extends CoordinatorShard<U>, U> {
|
|||
CoordinatorTimer<Void, U> timer
|
||||
);
|
||||
|
||||
/**
|
||||
* Sets the coordinator executor.
|
||||
*
|
||||
* @param executor The coordinator executor.
|
||||
*
|
||||
* @return The builder.
|
||||
*/
|
||||
CoordinatorShardBuilder<S, U> withExecutor(
|
||||
CoordinatorExecutor<U> executor
|
||||
);
|
||||
|
||||
/**
|
||||
* Sets the coordinator metrics.
|
||||
*
|
||||
|
|
|
@ -0,0 +1,317 @@
|
|||
/*
|
||||
* 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.utils.LogContext;
|
||||
import org.apache.kafka.server.util.FutureUtils;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.Collections;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.RejectedExecutionException;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.ArgumentMatchers.eq;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
// Creating mocks of classes using generics creates unsafe assignment.
|
||||
@SuppressWarnings("unchecked")
|
||||
public class CoordinatorExecutorImplTest {
|
||||
private static final LogContext LOG_CONTEXT = new LogContext();
|
||||
private static final TopicPartition SHARD_PARTITION = new TopicPartition("__consumer_offsets", 0);
|
||||
private static final Duration WRITE_TIMEOUT = Duration.ofMillis(1000);
|
||||
private static final String TASK_KEY = "task";
|
||||
|
||||
@Test
|
||||
public void testTaskSuccessfulLifecycle() {
|
||||
CoordinatorShard<String> coordinatorShard = mock(CoordinatorShard.class);
|
||||
CoordinatorRuntime<CoordinatorShard<String>, String> runtime = mock(CoordinatorRuntime.class);
|
||||
ExecutorService executorService = mock(ExecutorService.class);
|
||||
CoordinatorExecutorImpl<CoordinatorShard<String>, String> executor = new CoordinatorExecutorImpl<>(
|
||||
LOG_CONTEXT,
|
||||
SHARD_PARTITION,
|
||||
runtime,
|
||||
executorService,
|
||||
WRITE_TIMEOUT
|
||||
);
|
||||
|
||||
when(runtime.scheduleWriteOperation(
|
||||
eq(TASK_KEY),
|
||||
eq(SHARD_PARTITION),
|
||||
eq(WRITE_TIMEOUT),
|
||||
any()
|
||||
)).thenAnswer(args -> {
|
||||
assertTrue(executor.isScheduled(TASK_KEY));
|
||||
CoordinatorRuntime.CoordinatorWriteOperation<CoordinatorShard<String>, Void, String> op =
|
||||
args.getArgument(3);
|
||||
assertEquals(
|
||||
new CoordinatorResult<>(Collections.singletonList("record"), null),
|
||||
op.generateRecordsAndResult(coordinatorShard)
|
||||
);
|
||||
return CompletableFuture.completedFuture(null);
|
||||
});
|
||||
|
||||
when(executorService.submit(any(Runnable.class))).thenAnswer(args -> {
|
||||
assertTrue(executor.isScheduled(TASK_KEY));
|
||||
Runnable op = args.getArgument(0);
|
||||
op.run();
|
||||
return CompletableFuture.completedFuture(null);
|
||||
});
|
||||
|
||||
AtomicBoolean taskCalled = new AtomicBoolean(false);
|
||||
CoordinatorExecutor.TaskRunnable<String> taskRunnable = () -> {
|
||||
taskCalled.set(true);
|
||||
return "Hello!";
|
||||
};
|
||||
|
||||
AtomicBoolean operationCalled = new AtomicBoolean(false);
|
||||
CoordinatorExecutor.TaskOperation<String, String> taskOperation = (result, exception) -> {
|
||||
operationCalled.set(true);
|
||||
assertEquals("Hello!", result);
|
||||
assertNull(exception);
|
||||
return new CoordinatorResult<>(Collections.singletonList("record"), null);
|
||||
};
|
||||
|
||||
executor.schedule(
|
||||
TASK_KEY,
|
||||
taskRunnable,
|
||||
taskOperation
|
||||
);
|
||||
|
||||
assertTrue(taskCalled.get());
|
||||
assertTrue(operationCalled.get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskFailedLifecycle() {
|
||||
CoordinatorShard<String> coordinatorShard = mock(CoordinatorShard.class);
|
||||
CoordinatorRuntime<CoordinatorShard<String>, String> runtime = mock(CoordinatorRuntime.class);
|
||||
ExecutorService executorService = mock(ExecutorService.class);
|
||||
CoordinatorExecutorImpl<CoordinatorShard<String>, String> executor = new CoordinatorExecutorImpl<>(
|
||||
LOG_CONTEXT,
|
||||
SHARD_PARTITION,
|
||||
runtime,
|
||||
executorService,
|
||||
WRITE_TIMEOUT
|
||||
);
|
||||
|
||||
when(runtime.scheduleWriteOperation(
|
||||
eq(TASK_KEY),
|
||||
eq(SHARD_PARTITION),
|
||||
eq(WRITE_TIMEOUT),
|
||||
any()
|
||||
)).thenAnswer(args -> {
|
||||
CoordinatorRuntime.CoordinatorWriteOperation<CoordinatorShard<String>, Void, String> op =
|
||||
args.getArgument(3);
|
||||
assertEquals(
|
||||
new CoordinatorResult<>(Collections.emptyList(), null),
|
||||
op.generateRecordsAndResult(coordinatorShard)
|
||||
);
|
||||
return CompletableFuture.completedFuture(null);
|
||||
});
|
||||
|
||||
when(executorService.submit(any(Runnable.class))).thenAnswer(args -> {
|
||||
Runnable op = args.getArgument(0);
|
||||
op.run();
|
||||
return CompletableFuture.completedFuture(null);
|
||||
});
|
||||
|
||||
AtomicBoolean taskCalled = new AtomicBoolean(false);
|
||||
CoordinatorExecutor.TaskRunnable<String> taskRunnable = () -> {
|
||||
taskCalled.set(true);
|
||||
throw new Exception("Oh no!");
|
||||
};
|
||||
|
||||
AtomicBoolean operationCalled = new AtomicBoolean(false);
|
||||
CoordinatorExecutor.TaskOperation<String, String> taskOperation = (result, exception) -> {
|
||||
operationCalled.set(true);
|
||||
assertNull(result);
|
||||
assertNotNull(exception);
|
||||
assertEquals("Oh no!", exception.getMessage());
|
||||
return new CoordinatorResult<>(Collections.emptyList(), null);
|
||||
};
|
||||
|
||||
executor.schedule(
|
||||
TASK_KEY,
|
||||
taskRunnable,
|
||||
taskOperation
|
||||
);
|
||||
|
||||
assertTrue(taskCalled.get());
|
||||
assertTrue(operationCalled.get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskCancelledBeforeBeingExecuted() {
|
||||
CoordinatorRuntime<CoordinatorShard<String>, String> runtime = mock(CoordinatorRuntime.class);
|
||||
ExecutorService executorService = mock(ExecutorService.class);
|
||||
CoordinatorExecutorImpl<CoordinatorShard<String>, String> executor = new CoordinatorExecutorImpl<>(
|
||||
LOG_CONTEXT,
|
||||
SHARD_PARTITION,
|
||||
runtime,
|
||||
executorService,
|
||||
WRITE_TIMEOUT
|
||||
);
|
||||
|
||||
when(executorService.submit(any(Runnable.class))).thenAnswer(args -> {
|
||||
// Cancel the task before running it.
|
||||
executor.cancel(TASK_KEY);
|
||||
|
||||
// Running the task.
|
||||
Runnable op = args.getArgument(0);
|
||||
op.run();
|
||||
return CompletableFuture.completedFuture(null);
|
||||
});
|
||||
|
||||
AtomicBoolean taskCalled = new AtomicBoolean(false);
|
||||
CoordinatorExecutor.TaskRunnable<String> taskRunnable = () -> {
|
||||
taskCalled.set(true);
|
||||
return null;
|
||||
};
|
||||
|
||||
AtomicBoolean operationCalled = new AtomicBoolean(false);
|
||||
CoordinatorExecutor.TaskOperation<String, String> taskOperation = (result, exception) -> {
|
||||
operationCalled.set(true);
|
||||
return null;
|
||||
};
|
||||
|
||||
executor.schedule(
|
||||
TASK_KEY,
|
||||
taskRunnable,
|
||||
taskOperation
|
||||
);
|
||||
|
||||
assertFalse(taskCalled.get());
|
||||
assertFalse(operationCalled.get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskCancelledAfterBeingExecutedButBeforeWriteOperationIsExecuted() {
|
||||
CoordinatorShard<String> coordinatorShard = mock(CoordinatorShard.class);
|
||||
CoordinatorRuntime<CoordinatorShard<String>, String> runtime = mock(CoordinatorRuntime.class);
|
||||
ExecutorService executorService = mock(ExecutorService.class);
|
||||
CoordinatorExecutorImpl<CoordinatorShard<String>, String> executor = new CoordinatorExecutorImpl<>(
|
||||
LOG_CONTEXT,
|
||||
SHARD_PARTITION,
|
||||
runtime,
|
||||
executorService,
|
||||
WRITE_TIMEOUT
|
||||
);
|
||||
|
||||
when(runtime.scheduleWriteOperation(
|
||||
eq(TASK_KEY),
|
||||
eq(SHARD_PARTITION),
|
||||
eq(WRITE_TIMEOUT),
|
||||
any()
|
||||
)).thenAnswer(args -> {
|
||||
// Cancel the task before running the write operation.
|
||||
executor.cancel(TASK_KEY);
|
||||
|
||||
CoordinatorRuntime.CoordinatorWriteOperation<CoordinatorShard<String>, Void, String> op =
|
||||
args.getArgument(3);
|
||||
Throwable ex = assertThrows(RejectedExecutionException.class, () -> op.generateRecordsAndResult(coordinatorShard));
|
||||
return FutureUtils.failedFuture(ex);
|
||||
});
|
||||
|
||||
when(executorService.submit(any(Runnable.class))).thenAnswer(args -> {
|
||||
Runnable op = args.getArgument(0);
|
||||
op.run();
|
||||
return CompletableFuture.completedFuture(null);
|
||||
});
|
||||
|
||||
AtomicBoolean taskCalled = new AtomicBoolean(false);
|
||||
CoordinatorExecutor.TaskRunnable<String> taskRunnable = () -> {
|
||||
taskCalled.set(true);
|
||||
return "Hello!";
|
||||
};
|
||||
|
||||
AtomicBoolean operationCalled = new AtomicBoolean(false);
|
||||
CoordinatorExecutor.TaskOperation<String, String> taskOperation = (result, exception) -> {
|
||||
operationCalled.set(true);
|
||||
return null;
|
||||
};
|
||||
|
||||
executor.schedule(
|
||||
TASK_KEY,
|
||||
taskRunnable,
|
||||
taskOperation
|
||||
);
|
||||
|
||||
assertTrue(taskCalled.get());
|
||||
assertFalse(operationCalled.get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskSchedulingWriteOperationFailed() {
|
||||
CoordinatorRuntime<CoordinatorShard<String>, String> runtime = mock(CoordinatorRuntime.class);
|
||||
ExecutorService executorService = mock(ExecutorService.class);
|
||||
CoordinatorExecutorImpl<CoordinatorShard<String>, String> executor = new CoordinatorExecutorImpl<>(
|
||||
LOG_CONTEXT,
|
||||
SHARD_PARTITION,
|
||||
runtime,
|
||||
executorService,
|
||||
WRITE_TIMEOUT
|
||||
);
|
||||
|
||||
when(runtime.scheduleWriteOperation(
|
||||
eq(TASK_KEY),
|
||||
eq(SHARD_PARTITION),
|
||||
eq(WRITE_TIMEOUT),
|
||||
any()
|
||||
)).thenReturn(FutureUtils.failedFuture(new Throwable("Oh no!")));
|
||||
|
||||
when(executorService.submit(any(Runnable.class))).thenAnswer(args -> {
|
||||
Runnable op = args.getArgument(0);
|
||||
op.run();
|
||||
return CompletableFuture.completedFuture(null);
|
||||
});
|
||||
|
||||
AtomicBoolean taskCalled = new AtomicBoolean(false);
|
||||
CoordinatorExecutor.TaskRunnable<String> taskRunnable = () -> {
|
||||
taskCalled.set(true);
|
||||
return "Hello!";
|
||||
};
|
||||
|
||||
AtomicBoolean operationCalled = new AtomicBoolean(false);
|
||||
CoordinatorExecutor.TaskOperation<String, String> taskOperation = (result, exception) -> {
|
||||
operationCalled.set(true);
|
||||
return new CoordinatorResult<>(Collections.emptyList(), null);
|
||||
};
|
||||
|
||||
executor.schedule(
|
||||
TASK_KEY,
|
||||
taskRunnable,
|
||||
taskOperation
|
||||
);
|
||||
|
||||
assertTrue(taskCalled.get());
|
||||
assertFalse(operationCalled.get());
|
||||
assertFalse(executor.isScheduled(TASK_KEY));
|
||||
}
|
||||
}
|
|
@ -69,6 +69,7 @@ import java.util.OptionalInt;
|
|||
import java.util.Set;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.RejectedExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
@ -342,7 +343,7 @@ public class CoordinatorRuntimeTest {
|
|||
"offset=" + offset +
|
||||
", producerId=" + producerId +
|
||||
", producerEpoch=" + producerEpoch +
|
||||
", record='" + record.substring(0, 10) + '\'' +
|
||||
", record='" + record.substring(0, Math.min(10, record.length())) + '\'' +
|
||||
')';
|
||||
}
|
||||
}
|
||||
|
@ -351,15 +352,25 @@ public class CoordinatorRuntimeTest {
|
|||
private final TimelineHashSet<RecordAndMetadata> records;
|
||||
private final TimelineHashMap<Long, TimelineHashSet<RecordAndMetadata>> pendingRecords;
|
||||
private final CoordinatorTimer<Void, String> timer;
|
||||
private final CoordinatorExecutor<String> executor;
|
||||
|
||||
MockCoordinatorShard(
|
||||
SnapshotRegistry snapshotRegistry,
|
||||
CoordinatorTimer<Void, String> timer
|
||||
) {
|
||||
this(snapshotRegistry, timer, null);
|
||||
}
|
||||
|
||||
MockCoordinatorShard(
|
||||
SnapshotRegistry snapshotRegistry,
|
||||
CoordinatorTimer<Void, String> timer,
|
||||
CoordinatorExecutor<String> executor
|
||||
) {
|
||||
this.snapshotRegistry = snapshotRegistry;
|
||||
this.records = new TimelineHashSet<>(snapshotRegistry, 0);
|
||||
this.pendingRecords = new TimelineHashMap<>(snapshotRegistry, 0);
|
||||
this.timer = timer;
|
||||
this.executor = executor;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -424,6 +435,7 @@ public class CoordinatorRuntimeTest {
|
|||
private static class MockCoordinatorShardBuilder implements CoordinatorShardBuilder<MockCoordinatorShard, String> {
|
||||
private SnapshotRegistry snapshotRegistry;
|
||||
private CoordinatorTimer<Void, String> timer;
|
||||
private CoordinatorExecutor<String> executor;
|
||||
|
||||
@Override
|
||||
public CoordinatorShardBuilder<MockCoordinatorShard, String> withSnapshotRegistry(
|
||||
|
@ -447,6 +459,14 @@ public class CoordinatorRuntimeTest {
|
|||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CoordinatorShardBuilder<MockCoordinatorShard, String> withExecutor(
|
||||
CoordinatorExecutor<String> executor
|
||||
) {
|
||||
this.executor = executor;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CoordinatorShardBuilder<MockCoordinatorShard, String> withTimer(
|
||||
CoordinatorTimer<Void, String> timer
|
||||
|
@ -471,7 +491,8 @@ public class CoordinatorRuntimeTest {
|
|||
public MockCoordinatorShard build() {
|
||||
return new MockCoordinatorShard(
|
||||
Objects.requireNonNull(this.snapshotRegistry),
|
||||
Objects.requireNonNull(this.timer)
|
||||
Objects.requireNonNull(this.timer),
|
||||
Objects.requireNonNull(this.executor)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
@ -624,6 +645,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
when(builder.withSnapshotRegistry(any())).thenReturn(builder);
|
||||
|
@ -632,6 +654,7 @@ public class CoordinatorRuntimeTest {
|
|||
when(builder.withTimer(any())).thenReturn(builder);
|
||||
when(builder.withCoordinatorMetrics(any())).thenReturn(builder);
|
||||
when(builder.withTopicPartition(any())).thenReturn(builder);
|
||||
when(builder.withExecutor(any())).thenReturn(builder);
|
||||
when(builder.build()).thenReturn(coordinator);
|
||||
when(supplier.get()).thenReturn(builder);
|
||||
CompletableFuture<CoordinatorLoader.LoadSummary> future = new CompletableFuture<>();
|
||||
|
@ -694,6 +717,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
when(builder.withSnapshotRegistry(any())).thenReturn(builder);
|
||||
|
@ -702,6 +726,7 @@ public class CoordinatorRuntimeTest {
|
|||
when(builder.withTimer(any())).thenReturn(builder);
|
||||
when(builder.withCoordinatorMetrics(any())).thenReturn(builder);
|
||||
when(builder.withTopicPartition(any())).thenReturn(builder);
|
||||
when(builder.withExecutor(any())).thenReturn(builder);
|
||||
when(builder.build()).thenReturn(coordinator);
|
||||
when(supplier.get()).thenReturn(builder);
|
||||
CompletableFuture<CoordinatorLoader.LoadSummary> future = new CompletableFuture<>();
|
||||
|
@ -744,6 +769,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
when(builder.withSnapshotRegistry(any())).thenReturn(builder);
|
||||
|
@ -752,6 +778,7 @@ public class CoordinatorRuntimeTest {
|
|||
when(builder.withTimer(any())).thenReturn(builder);
|
||||
when(builder.withCoordinatorMetrics(any())).thenReturn(builder);
|
||||
when(builder.withTopicPartition(any())).thenReturn(builder);
|
||||
when(builder.withExecutor(any())).thenReturn(builder);
|
||||
when(builder.build()).thenReturn(coordinator);
|
||||
when(supplier.get()).thenReturn(builder);
|
||||
CompletableFuture<CoordinatorLoader.LoadSummary> future = new CompletableFuture<>();
|
||||
|
@ -798,6 +825,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
when(builder.withSnapshotRegistry(any())).thenReturn(builder);
|
||||
|
@ -806,6 +834,7 @@ public class CoordinatorRuntimeTest {
|
|||
when(builder.withTimer(any())).thenReturn(builder);
|
||||
when(builder.withCoordinatorMetrics(any())).thenReturn(builder);
|
||||
when(builder.withTopicPartition(any())).thenReturn(builder);
|
||||
when(builder.withExecutor(any())).thenReturn(builder);
|
||||
when(builder.build()).thenReturn(coordinator);
|
||||
when(supplier.get()).thenReturn(builder);
|
||||
CompletableFuture<CoordinatorLoader.LoadSummary> future = new CompletableFuture<>();
|
||||
|
@ -869,6 +898,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
when(builder.withSnapshotRegistry(any())).thenReturn(builder);
|
||||
|
@ -877,6 +907,7 @@ public class CoordinatorRuntimeTest {
|
|||
when(builder.withTimer(any())).thenReturn(builder);
|
||||
when(builder.withCoordinatorMetrics(any())).thenReturn(builder);
|
||||
when(builder.withTopicPartition(any())).thenReturn(builder);
|
||||
when(builder.withExecutor(any())).thenReturn(builder);
|
||||
when(builder.build()).thenReturn(coordinator);
|
||||
when(supplier.get()).thenReturn(builder);
|
||||
|
||||
|
@ -923,6 +954,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
when(builder.withSnapshotRegistry(any())).thenReturn(builder);
|
||||
|
@ -931,6 +963,7 @@ public class CoordinatorRuntimeTest {
|
|||
when(builder.withTimer(any())).thenReturn(builder);
|
||||
when(builder.withCoordinatorMetrics(any())).thenReturn(builder);
|
||||
when(builder.withTopicPartition(any())).thenReturn(builder);
|
||||
when(builder.withExecutor(any())).thenReturn(builder);
|
||||
when(builder.build()).thenReturn(coordinator);
|
||||
when(supplier.get()).thenReturn(builder);
|
||||
|
||||
|
@ -977,6 +1010,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
when(builder.withSnapshotRegistry(any())).thenReturn(builder);
|
||||
|
@ -984,6 +1018,7 @@ public class CoordinatorRuntimeTest {
|
|||
when(builder.withTime(any())).thenReturn(builder);
|
||||
when(builder.withTimer(any())).thenReturn(builder);
|
||||
when(builder.withTopicPartition(any())).thenReturn(builder);
|
||||
when(builder.withExecutor(any())).thenReturn(builder);
|
||||
when(builder.build()).thenReturn(coordinator);
|
||||
when(supplier.get()).thenReturn(builder);
|
||||
|
||||
|
@ -1019,6 +1054,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
when(builder.withSnapshotRegistry(any())).thenReturn(builder);
|
||||
|
@ -1029,6 +1065,8 @@ public class CoordinatorRuntimeTest {
|
|||
when(builder.withTimer(any())).thenReturn(builder);
|
||||
when(builder.withCoordinatorMetrics(any())).thenReturn(builder);
|
||||
when(builder.withTopicPartition(any())).thenReturn(builder);
|
||||
when(builder.withExecutor(any())).thenReturn(builder);
|
||||
when(builder.withExecutor(any())).thenReturn(builder);
|
||||
when(builder.build()).thenReturn(coordinator);
|
||||
when(supplier.get()).thenReturn(builder);
|
||||
|
||||
|
@ -1062,6 +1100,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Schedule the loading.
|
||||
|
@ -1183,6 +1222,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Scheduling a write fails with a NotCoordinatorException because the coordinator
|
||||
|
@ -1207,6 +1247,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Loads the coordinator.
|
||||
|
@ -1235,6 +1276,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Loads the coordinator.
|
||||
|
@ -1295,6 +1337,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Loads the coordinator.
|
||||
|
@ -1347,6 +1390,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Loads the coordinator.
|
||||
|
@ -1384,6 +1428,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
TopicPartition coordinator0 = new TopicPartition("__consumer_offsets", 0);
|
||||
|
@ -1455,6 +1500,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Schedule the loading.
|
||||
|
@ -1548,6 +1594,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Schedule the loading.
|
||||
|
@ -1606,6 +1653,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Schedule the loading.
|
||||
|
@ -1718,6 +1766,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Loads the coordinator.
|
||||
|
@ -1775,6 +1824,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Loads the coordinator.
|
||||
|
@ -1841,6 +1891,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Loads the coordinator.
|
||||
|
@ -1933,6 +1984,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Loads the coordinator.
|
||||
|
@ -1991,6 +2043,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Schedule a read. It fails because the coordinator does not exist.
|
||||
|
@ -2016,6 +2069,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Loads the coordinator.
|
||||
|
@ -2062,6 +2116,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
TopicPartition coordinator0 = new TopicPartition("__consumer_offsets", 0);
|
||||
|
@ -2102,6 +2157,7 @@ public class CoordinatorRuntimeTest {
|
|||
public void testClose() throws Exception {
|
||||
MockCoordinatorLoader loader = spy(new MockCoordinatorLoader());
|
||||
MockTimer timer = new MockTimer();
|
||||
ExecutorService executorService = mock(ExecutorService.class);
|
||||
CoordinatorRuntime<MockCoordinatorShard, String> runtime =
|
||||
new CoordinatorRuntime.Builder<MockCoordinatorShard, String>()
|
||||
.withTime(timer.time())
|
||||
|
@ -2114,6 +2170,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(executorService)
|
||||
.build();
|
||||
|
||||
// Loads the coordinator.
|
||||
|
@ -2158,6 +2215,9 @@ public class CoordinatorRuntimeTest {
|
|||
|
||||
// The coordinator timer should be empty.
|
||||
assertEquals(0, ctx.timer.size());
|
||||
|
||||
// Verify that the executor service was shutdown.
|
||||
verify(executorService).shutdown();
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -2183,6 +2243,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
MockCoordinatorShard coordinator0 = mock(MockCoordinatorShard.class);
|
||||
|
@ -2195,6 +2256,7 @@ public class CoordinatorRuntimeTest {
|
|||
when(builder.withTimer(any())).thenReturn(builder);
|
||||
when(builder.withCoordinatorMetrics(any())).thenReturn(builder);
|
||||
when(builder.withTopicPartition(any())).thenReturn(builder);
|
||||
when(builder.withExecutor(any())).thenReturn(builder);
|
||||
when(builder.withTime(any())).thenReturn(builder);
|
||||
when(builder.build())
|
||||
.thenReturn(coordinator0)
|
||||
|
@ -2246,6 +2308,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Loads the coordinator.
|
||||
|
@ -2301,6 +2364,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Loads the coordinator.
|
||||
|
@ -2376,6 +2440,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Loads the coordinator.
|
||||
|
@ -2448,6 +2513,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Loads the coordinator.
|
||||
|
@ -2508,6 +2574,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Loads the coordinator.
|
||||
|
@ -2582,6 +2649,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Loads the coordinator.
|
||||
|
@ -2625,6 +2693,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Loads the coordinator.
|
||||
|
@ -2683,6 +2752,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(runtimeMetrics)
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
when(builder.withSnapshotRegistry(any())).thenReturn(builder);
|
||||
|
@ -2691,6 +2761,7 @@ public class CoordinatorRuntimeTest {
|
|||
when(builder.withTimer(any())).thenReturn(builder);
|
||||
when(builder.withCoordinatorMetrics(any())).thenReturn(builder);
|
||||
when(builder.withTopicPartition(any())).thenReturn(builder);
|
||||
when(builder.withExecutor(any())).thenReturn(builder);
|
||||
when(builder.build()).thenReturn(coordinator);
|
||||
when(supplier.get()).thenReturn(builder);
|
||||
CompletableFuture<CoordinatorLoader.LoadSummary> future = new CompletableFuture<>();
|
||||
|
@ -2761,6 +2832,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(runtimeMetrics)
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
when(builder.withSnapshotRegistry(any())).thenReturn(builder);
|
||||
|
@ -2769,6 +2841,7 @@ public class CoordinatorRuntimeTest {
|
|||
when(builder.withTimer(any())).thenReturn(builder);
|
||||
when(builder.withCoordinatorMetrics(any())).thenReturn(builder);
|
||||
when(builder.withTopicPartition(any())).thenReturn(builder);
|
||||
when(builder.withExecutor(any())).thenReturn(builder);
|
||||
when(builder.build()).thenReturn(coordinator);
|
||||
when(supplier.get()).thenReturn(builder);
|
||||
|
||||
|
@ -2817,6 +2890,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(runtimeMetrics)
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
when(builder.withSnapshotRegistry(any())).thenReturn(builder);
|
||||
|
@ -2825,6 +2899,7 @@ public class CoordinatorRuntimeTest {
|
|||
when(builder.withTimer(any())).thenReturn(builder);
|
||||
when(builder.withCoordinatorMetrics(any())).thenReturn(builder);
|
||||
when(builder.withTopicPartition(any())).thenReturn(builder);
|
||||
when(builder.withExecutor(any())).thenReturn(builder);
|
||||
when(builder.build()).thenReturn(coordinator);
|
||||
when(supplier.get()).thenReturn(builder);
|
||||
|
||||
|
@ -2874,6 +2949,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(runtimeMetrics)
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
when(builder.withSnapshotRegistry(any())).thenReturn(builder);
|
||||
|
@ -2882,6 +2958,7 @@ public class CoordinatorRuntimeTest {
|
|||
when(builder.withTimer(any())).thenReturn(builder);
|
||||
when(builder.withCoordinatorMetrics(any())).thenReturn(builder);
|
||||
when(builder.withTopicPartition(any())).thenReturn(builder);
|
||||
when(builder.withExecutor(any())).thenReturn(builder);
|
||||
when(builder.build()).thenReturn(coordinator);
|
||||
when(supplier.get()).thenReturn(builder);
|
||||
|
||||
|
@ -2917,6 +2994,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Loads the coordinator. Poll once to execute the load operation and once
|
||||
|
@ -2987,6 +3065,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Loads the coordinator. Poll once to execute the load operation and once
|
||||
|
@ -3061,6 +3140,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Loads the coordinator. Poll once to execute the load operation and once
|
||||
|
@ -3130,6 +3210,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class))
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(serializer)
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Schedule the loading.
|
||||
|
@ -3181,6 +3262,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withAppendLingerMs(10)
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Schedule the loading.
|
||||
|
@ -3315,6 +3397,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withAppendLingerMs(10)
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Schedule the loading.
|
||||
|
@ -3366,6 +3449,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withAppendLingerMs(10)
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Schedule the loading.
|
||||
|
@ -3451,6 +3535,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withAppendLingerMs(10)
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Schedule the loading.
|
||||
|
@ -3548,6 +3633,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withAppendLingerMs(10)
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Schedule the loading.
|
||||
|
@ -3690,6 +3776,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withAppendLingerMs(10)
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Schedule the loading.
|
||||
|
@ -3804,6 +3891,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withAppendLingerMs(10)
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Schedule the loading.
|
||||
|
@ -3852,6 +3940,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withAppendLingerMs(10)
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Schedule the loading.
|
||||
|
@ -3960,6 +4049,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withAppendLingerMs(10)
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Schedule the loading.
|
||||
|
@ -4055,6 +4145,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withAppendLingerMs(10)
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Schedule the loading.
|
||||
|
@ -4143,6 +4234,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withAppendLingerMs(10)
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Schedule the loading.
|
||||
|
@ -4253,6 +4345,7 @@ public class CoordinatorRuntimeTest {
|
|||
.withCoordinatorRuntimeMetrics(runtimeMetrics)
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(mock(ExecutorService.class))
|
||||
.build();
|
||||
|
||||
// Loads the coordinator. Poll once to execute the load operation and once
|
||||
|
@ -4317,6 +4410,93 @@ public class CoordinatorRuntimeTest {
|
|||
verify(runtimeMetrics, times(1)).recordEventPurgatoryTime(writeTimeout.toMillis() + 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCoordinatorExecutor() {
|
||||
Duration writeTimeout = Duration.ofMillis(1000);
|
||||
MockTimer timer = new MockTimer();
|
||||
MockPartitionWriter writer = new MockPartitionWriter();
|
||||
ManualEventProcessor processor = new ManualEventProcessor();
|
||||
CoordinatorRuntimeMetrics runtimeMetrics = mock(CoordinatorRuntimeMetrics.class);
|
||||
ExecutorService executorService = mock(ExecutorService.class);
|
||||
|
||||
when(executorService.submit(any(Runnable.class))).thenAnswer(args -> {
|
||||
Runnable op = args.getArgument(0);
|
||||
op.run();
|
||||
return CompletableFuture.completedFuture(null);
|
||||
});
|
||||
|
||||
CoordinatorRuntime<MockCoordinatorShard, String> runtime =
|
||||
new CoordinatorRuntime.Builder<MockCoordinatorShard, String>()
|
||||
.withTime(timer.time())
|
||||
.withTimer(timer)
|
||||
.withDefaultWriteTimeOut(writeTimeout)
|
||||
.withLoader(new MockCoordinatorLoader())
|
||||
.withEventProcessor(processor)
|
||||
.withPartitionWriter(writer)
|
||||
.withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier())
|
||||
.withCoordinatorRuntimeMetrics(runtimeMetrics)
|
||||
.withCoordinatorMetrics(mock(CoordinatorMetrics.class))
|
||||
.withSerializer(new StringSerializer())
|
||||
.withExecutorService(executorService)
|
||||
.build();
|
||||
|
||||
// Loads the coordinator. Poll once to execute the load operation and once
|
||||
// to complete the load.
|
||||
runtime.scheduleLoadOperation(TP, 10);
|
||||
processor.poll();
|
||||
processor.poll();
|
||||
|
||||
// Schedule a write which schedules an async tasks.
|
||||
CompletableFuture<String> write1 = runtime.scheduleWriteOperation("write#1", TP, writeTimeout,
|
||||
state -> {
|
||||
state.executor.schedule(
|
||||
"write#1#task",
|
||||
() -> "task result",
|
||||
(result, exception) -> {
|
||||
assertEquals("task result", result);
|
||||
assertNull(exception);
|
||||
return new CoordinatorResult<>(Collections.singletonList("record2"), null);
|
||||
}
|
||||
);
|
||||
return new CoordinatorResult<>(Collections.singletonList("record1"), "response1");
|
||||
}
|
||||
);
|
||||
|
||||
// Execute the write.
|
||||
processor.poll();
|
||||
|
||||
// We should have a new write event in the queue as a result of the
|
||||
// task being executed immediately.
|
||||
assertEquals(1, processor.queue.size());
|
||||
|
||||
// Verify the state.
|
||||
CoordinatorRuntime<MockCoordinatorShard, String>.CoordinatorContext ctx = runtime.contextOrThrow(TP);
|
||||
assertEquals(1L, ctx.coordinator.lastWrittenOffset());
|
||||
assertEquals(0L, ctx.coordinator.lastCommittedOffset());
|
||||
assertEquals(List.of(
|
||||
new MockCoordinatorShard.RecordAndMetadata(0, "record1")
|
||||
), ctx.coordinator.coordinator().fullRecords());
|
||||
|
||||
// Execute the pending write.
|
||||
processor.poll();
|
||||
|
||||
// The processor must be empty now.
|
||||
assertEquals(0, processor.queue.size());
|
||||
|
||||
// Verify the state.
|
||||
assertEquals(2L, ctx.coordinator.lastWrittenOffset());
|
||||
assertEquals(0L, ctx.coordinator.lastCommittedOffset());
|
||||
assertEquals(List.of(
|
||||
new MockCoordinatorShard.RecordAndMetadata(0, "record1"),
|
||||
new MockCoordinatorShard.RecordAndMetadata(1, "record2")
|
||||
), ctx.coordinator.coordinator().fullRecords());
|
||||
|
||||
// Commit.
|
||||
writer.commit(TP);
|
||||
processor.poll();
|
||||
assertTrue(write1.isDone());
|
||||
}
|
||||
|
||||
private static <S extends CoordinatorShard<U>, U> ArgumentMatcher<CoordinatorPlayback<U>> coordinatorMatcher(
|
||||
CoordinatorRuntime<S, U> runtime,
|
||||
TopicPartition tp
|
||||
|
|
|
@ -92,6 +92,7 @@ import java.util.Properties;
|
|||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionException;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.function.IntSupplier;
|
||||
import java.util.stream.Collectors;
|
||||
|
@ -205,6 +206,7 @@ public class GroupCoordinatorService implements GroupCoordinator {
|
|||
.withSerializer(new GroupCoordinatorRecordSerde())
|
||||
.withCompression(Compression.of(config.offsetTopicCompressionType()).build())
|
||||
.withAppendLingerMs(config.appendLingerMs())
|
||||
.withExecutorService(Executors.newSingleThreadExecutor())
|
||||
.build();
|
||||
|
||||
return new GroupCoordinatorService(
|
||||
|
|
|
@ -48,6 +48,7 @@ import org.apache.kafka.common.requests.RequestContext;
|
|||
import org.apache.kafka.common.requests.TransactionResult;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.coordinator.common.runtime.CoordinatorExecutor;
|
||||
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetrics;
|
||||
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetricsShard;
|
||||
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
|
||||
|
@ -121,6 +122,7 @@ public class GroupCoordinatorShard implements CoordinatorShard<CoordinatorRecord
|
|||
private SnapshotRegistry snapshotRegistry;
|
||||
private Time time;
|
||||
private CoordinatorTimer<Void, CoordinatorRecord> timer;
|
||||
private CoordinatorExecutor<CoordinatorRecord> executor;
|
||||
private CoordinatorMetrics coordinatorMetrics;
|
||||
private TopicPartition topicPartition;
|
||||
|
||||
|
@ -156,6 +158,14 @@ public class GroupCoordinatorShard implements CoordinatorShard<CoordinatorRecord
|
|||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CoordinatorShardBuilder<GroupCoordinatorShard, CoordinatorRecord> withExecutor(
|
||||
CoordinatorExecutor<CoordinatorRecord> executor
|
||||
) {
|
||||
this.executor = executor;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CoordinatorShardBuilder<GroupCoordinatorShard, CoordinatorRecord> withCoordinatorMetrics(
|
||||
CoordinatorMetrics coordinatorMetrics
|
||||
|
@ -178,6 +188,7 @@ public class GroupCoordinatorShard implements CoordinatorShard<CoordinatorRecord
|
|||
return this;
|
||||
}
|
||||
|
||||
@SuppressWarnings("NPathComplexity")
|
||||
@Override
|
||||
public GroupCoordinatorShard build() {
|
||||
if (logContext == null) logContext = new LogContext();
|
||||
|
@ -189,6 +200,8 @@ public class GroupCoordinatorShard implements CoordinatorShard<CoordinatorRecord
|
|||
throw new IllegalArgumentException("Time must be set.");
|
||||
if (timer == null)
|
||||
throw new IllegalArgumentException("Timer must be set.");
|
||||
if (executor == null)
|
||||
throw new IllegalArgumentException("Executor must be set.");
|
||||
if (coordinatorMetrics == null || !(coordinatorMetrics instanceof GroupCoordinatorMetrics))
|
||||
throw new IllegalArgumentException("CoordinatorMetrics must be set and be of type GroupCoordinatorMetrics.");
|
||||
if (topicPartition == null)
|
||||
|
|
|
@ -60,6 +60,7 @@ import java.util.Map;
|
|||
import java.util.OptionalInt;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.function.IntSupplier;
|
||||
import java.util.stream.Collectors;
|
||||
|
@ -175,6 +176,7 @@ public class ShareCoordinatorService implements ShareCoordinator {
|
|||
.withSerializer(new ShareCoordinatorRecordSerde())
|
||||
.withCompression(Compression.of(config.shareCoordinatorStateTopicCompressionType()).build())
|
||||
.withAppendLingerMs(config.shareCoordinatorAppendLingerMs())
|
||||
.withExecutorService(Executors.newSingleThreadExecutor())
|
||||
.build();
|
||||
|
||||
return new ShareCoordinatorService(
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.kafka.common.requests.TransactionResult;
|
|||
import org.apache.kafka.common.requests.WriteShareGroupStateResponse;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.coordinator.common.runtime.CoordinatorExecutor;
|
||||
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetrics;
|
||||
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetricsShard;
|
||||
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
|
||||
|
@ -111,6 +112,12 @@ public class ShareCoordinatorShard implements CoordinatorShard<CoordinatorRecord
|
|||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CoordinatorShardBuilder<ShareCoordinatorShard, CoordinatorRecord> withExecutor(CoordinatorExecutor<CoordinatorRecord> executor) {
|
||||
// method is required due to interface
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CoordinatorShardBuilder<ShareCoordinatorShard, CoordinatorRecord> withCoordinatorMetrics(CoordinatorMetrics coordinatorMetrics) {
|
||||
this.coordinatorMetrics = coordinatorMetrics;
|
||||
|
|
Loading…
Reference in New Issue