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:
David Jacot 2024-11-19 17:19:22 +02:00 committed by GitHub
parent a334b1b6fd
commit a211ee99b5
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
10 changed files with 801 additions and 4 deletions

View File

@ -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);
}

View File

@ -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();
}
}
}

View File

@ -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.");
}

View File

@ -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.
*

View File

@ -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));
}
}

View File

@ -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

View File

@ -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(

View File

@ -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)

View File

@ -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(

View File

@ -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;