You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "jolshan (via GitHub)" <gi...@apache.org> on 2023/05/30 17:59:56 UTC

[GitHub] [kafka] jolshan commented on a diff in pull request #13666: KAFKA-14462; [13/N] CoordinatorEvent and CoordinatorEventProcessor

jolshan commented on code in PR #13666:
URL: https://github.com/apache/kafka/pull/13666#discussion_r1210611398


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorEventProcessor.java:
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.group.runtime;
+
+import java.util.concurrent.RejectedExecutionException;
+
+/**
+ * A {{@link CoordinatorEvent}} processor.
+ */
+public interface CoordinatorEventProcessor extends AutoCloseable {
+    /**
+     * Enqueues a new {{@link CoordinatorEvent}}.
+     *
+     * @param event The event.
+     * @throws RejectedExecutionException If the event processor. is closed.

Review Comment:
   nit: extra period



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.group.runtime;
+
+import org.apache.kafka.common.utils.LogContext;
+import org.slf4j.Logger;
+
+import java.util.List;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * A multithreaded {{@link CoordinatorEvent}} processor which uses a {{@link EventAccumulator}}
+ * which guarantees that events sharing a partition key are not processed concurrently.
+ */
+public class MultiThreadedEventProcessor implements CoordinatorEventProcessor {
+
+    /**
+     * The logger.
+     */
+    private final Logger log;
+
+    /**
+     * The accumulator.
+     */
+    private final EventAccumulator<Integer, CoordinatorEvent> accumulator;
+
+    /**
+     * The processing threads.
+     */
+    private final List<EventProcessorThread> threads;
+
+    /**
+     * The lock for protecting access to the resources.
+     */
+    private final ReentrantLock lock;
+
+    /**
+     * A boolean indicated whether the event processor is shutting down.
+     */
+    private volatile boolean shuttingDown;
+
+    /**
+     * Constructor.
+     *
+     * @param logContext    The log context.
+     * @param threadPrefix  The thread prefix.
+     * @param numThreads    The number of threads.
+     */
+    public MultiThreadedEventProcessor(
+        LogContext logContext,
+        String threadPrefix,
+        int numThreads
+    ) {
+        this.log = logContext.logger(MultiThreadedEventProcessor.class);
+        this.shuttingDown = false;
+        this.lock = new ReentrantLock();
+        this.accumulator = new EventAccumulator<>();
+        this.threads = IntStream.range(0, numThreads).mapToObj(threadId ->
+            new EventProcessorThread(
+                threadPrefix + threadId
+            )
+        ).collect(Collectors.toList());
+        this.threads.forEach(EventProcessorThread::start);
+    }
+
+    /**
+     * The event processor thread. The thread pulls events from the
+     * accumulator and runs them.
+     */
+    class EventProcessorThread extends Thread {
+        private final Logger log;
+
+        EventProcessorThread(
+            String name
+        ) {
+            super(name);
+            log = new LogContext("[" + name + "]: ").logger(EventProcessorThread.class);
+            setDaemon(false);
+        }
+
+        private void handleEvents() {
+            while (!shuttingDown) {
+                CoordinatorEvent event = accumulator.poll();
+                if (event != null) {
+                    try {
+                        log.debug("Executing event: {}.", event);
+                        event.run();
+                    } catch (Throwable t) {
+                        log.error("Failed to run event {} due to: {}.", event, t.getMessage(), t);
+                        event.complete(t);
+                    } finally {
+                        accumulator.done(event);
+                    }
+                }
+            }
+        }
+
+        private void drainEvents() {
+            CoordinatorEvent event = accumulator.poll(0, TimeUnit.MILLISECONDS);
+            while (event != null) {
+                try {
+                    log.debug("Draining event: {}.", event);
+                    event.complete(new RejectedExecutionException("EventProcessor is closed."));
+                } catch (Throwable t) {
+                    log.error("Failed to reject event {} due to: {}.", event, t.getMessage(), t);
+                } finally {
+                    accumulator.done(event);
+                }
+
+                event = accumulator.poll(0, TimeUnit.MILLISECONDS);
+            }
+        }
+
+        @Override
+        public void run() {
+            log.info("Starting");
+
+            try {
+                handleEvents();
+            } catch (Throwable t) {
+                log.error("Exiting with exception.", t);
+            }
+
+            log.info("Shutting down. Draining the remaining events.");
+
+            // The accumulator is drained and all the pending events are rejected
+            // when the event processor is shutdown.
+            if (shuttingDown) {
+                try {
+                    drainEvents();
+                } catch (Throwable t) {
+                    log.error("Draining threw exception.", t);
+                }
+            }
+
+            log.info("Shutdown completed");
+        }
+    }
+
+    /**
+     * Enqueues a new {{@link CoordinatorEvent}}.
+     *
+     * @param event The event.
+     * @throws RejectedExecutionException If the event processor. is closed.

Review Comment:
   same nit here about the extra "."



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.group.runtime;
+
+import org.apache.kafka.common.utils.LogContext;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import static org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS;
+import static org.apache.kafka.test.TestUtils.waitForCondition;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@Timeout(value = 60)
+public class MultiThreadedEventProcessorTest {
+
+    private static class FutureEvent<T> implements CoordinatorEvent {
+        private final int key;
+        private final CompletableFuture<T> future;
+        private final Supplier<T> supplier;
+
+        FutureEvent(
+            int key,
+            Supplier<T> supplier
+        ) {
+            this.key = key;
+            this.future = new CompletableFuture<>();
+            this.supplier = supplier;
+        }
+
+        @Override
+        public void run() {
+            future.complete(supplier.get());
+        }
+
+        @Override
+        public void complete(Throwable ex) {
+            future.completeExceptionally(ex);
+        }
+
+        @Override
+        public Integer key() {

Review Comment:
   Is there any implication for keys here in the test? Or just something we needed to override.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.group.runtime;
+
+import org.apache.kafka.common.utils.LogContext;
+import org.slf4j.Logger;
+
+import java.util.List;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * A multithreaded {{@link CoordinatorEvent}} processor which uses a {{@link EventAccumulator}}
+ * which guarantees that events sharing a partition key are not processed concurrently.
+ */
+public class MultiThreadedEventProcessor implements CoordinatorEventProcessor {
+
+    /**
+     * The logger.
+     */
+    private final Logger log;
+
+    /**
+     * The accumulator.
+     */
+    private final EventAccumulator<Integer, CoordinatorEvent> accumulator;
+
+    /**
+     * The processing threads.
+     */
+    private final List<EventProcessorThread> threads;
+
+    /**
+     * The lock for protecting access to the resources.
+     */
+    private final ReentrantLock lock;
+
+    /**
+     * A boolean indicated whether the event processor is shutting down.
+     */
+    private volatile boolean shuttingDown;
+
+    /**
+     * Constructor.
+     *
+     * @param logContext    The log context.
+     * @param threadPrefix  The thread prefix.
+     * @param numThreads    The number of threads.
+     */
+    public MultiThreadedEventProcessor(
+        LogContext logContext,
+        String threadPrefix,
+        int numThreads
+    ) {
+        this.log = logContext.logger(MultiThreadedEventProcessor.class);
+        this.shuttingDown = false;
+        this.lock = new ReentrantLock();
+        this.accumulator = new EventAccumulator<>();
+        this.threads = IntStream.range(0, numThreads).mapToObj(threadId ->
+            new EventProcessorThread(
+                threadPrefix + threadId
+            )
+        ).collect(Collectors.toList());
+        this.threads.forEach(EventProcessorThread::start);
+    }
+
+    /**
+     * The event processor thread. The thread pulls events from the
+     * accumulator and runs them.
+     */
+    class EventProcessorThread extends Thread {
+        private final Logger log;
+
+        EventProcessorThread(
+            String name
+        ) {
+            super(name);
+            log = new LogContext("[" + name + "]: ").logger(EventProcessorThread.class);
+            setDaemon(false);
+        }
+
+        private void handleEvents() {
+            while (!shuttingDown) {
+                CoordinatorEvent event = accumulator.poll();
+                if (event != null) {
+                    try {
+                        log.debug("Executing event: {}.", event);
+                        event.run();
+                    } catch (Throwable t) {
+                        log.error("Failed to run event {} due to: {}.", event, t.getMessage(), t);
+                        event.complete(t);
+                    } finally {
+                        accumulator.done(event);
+                    }
+                }
+            }
+        }
+
+        private void drainEvents() {
+            CoordinatorEvent event = accumulator.poll(0, TimeUnit.MILLISECONDS);
+            while (event != null) {
+                try {
+                    log.debug("Draining event: {}.", event);
+                    event.complete(new RejectedExecutionException("EventProcessor is closed."));
+                } catch (Throwable t) {
+                    log.error("Failed to reject event {} due to: {}.", event, t.getMessage(), t);
+                } finally {
+                    accumulator.done(event);
+                }
+
+                event = accumulator.poll(0, TimeUnit.MILLISECONDS);
+            }
+        }
+
+        @Override
+        public void run() {
+            log.info("Starting");
+
+            try {
+                handleEvents();
+            } catch (Throwable t) {
+                log.error("Exiting with exception.", t);
+            }
+
+            log.info("Shutting down. Draining the remaining events.");

Review Comment:
   Should this be outside the `if (shuttingDown)` block?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.group.runtime;
+
+import org.apache.kafka.common.utils.LogContext;
+import org.slf4j.Logger;
+
+import java.util.List;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * A multithreaded {{@link CoordinatorEvent}} processor which uses a {{@link EventAccumulator}}
+ * which guarantees that events sharing a partition key are not processed concurrently.
+ */
+public class MultiThreadedEventProcessor implements CoordinatorEventProcessor {
+
+    /**
+     * The logger.
+     */
+    private final Logger log;
+
+    /**
+     * The accumulator.
+     */
+    private final EventAccumulator<Integer, CoordinatorEvent> accumulator;
+
+    /**
+     * The processing threads.
+     */
+    private final List<EventProcessorThread> threads;
+
+    /**
+     * The lock for protecting access to the resources.
+     */
+    private final ReentrantLock lock;
+
+    /**
+     * A boolean indicated whether the event processor is shutting down.
+     */
+    private volatile boolean shuttingDown;
+
+    /**
+     * Constructor.
+     *
+     * @param logContext    The log context.
+     * @param threadPrefix  The thread prefix.
+     * @param numThreads    The number of threads.
+     */
+    public MultiThreadedEventProcessor(
+        LogContext logContext,
+        String threadPrefix,
+        int numThreads
+    ) {
+        this.log = logContext.logger(MultiThreadedEventProcessor.class);
+        this.shuttingDown = false;
+        this.lock = new ReentrantLock();
+        this.accumulator = new EventAccumulator<>();
+        this.threads = IntStream.range(0, numThreads).mapToObj(threadId ->
+            new EventProcessorThread(
+                threadPrefix + threadId
+            )
+        ).collect(Collectors.toList());
+        this.threads.forEach(EventProcessorThread::start);
+    }
+
+    /**
+     * The event processor thread. The thread pulls events from the
+     * accumulator and runs them.
+     */
+    class EventProcessorThread extends Thread {
+        private final Logger log;
+
+        EventProcessorThread(
+            String name
+        ) {
+            super(name);
+            log = new LogContext("[" + name + "]: ").logger(EventProcessorThread.class);
+            setDaemon(false);
+        }
+
+        private void handleEvents() {
+            while (!shuttingDown) {
+                CoordinatorEvent event = accumulator.poll();

Review Comment:
   I noticed we lock on enqueuing and shutdown, but not when we remove from the accumulator. 
   
   Is the main reason for the lock to prevent the queue from never emptying when shutting down?



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.group.runtime;
+
+import org.apache.kafka.common.utils.LogContext;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import static org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS;
+import static org.apache.kafka.test.TestUtils.waitForCondition;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@Timeout(value = 60)
+public class MultiThreadedEventProcessorTest {
+
+    private static class FutureEvent<T> implements CoordinatorEvent {
+        private final int key;
+        private final CompletableFuture<T> future;
+        private final Supplier<T> supplier;
+
+        FutureEvent(
+            int key,
+            Supplier<T> supplier
+        ) {
+            this.key = key;
+            this.future = new CompletableFuture<>();
+            this.supplier = supplier;
+        }
+
+        @Override
+        public void run() {
+            future.complete(supplier.get());
+        }
+
+        @Override
+        public void complete(Throwable ex) {
+            future.completeExceptionally(ex);
+        }
+
+        @Override
+        public Integer key() {
+            return key;
+        }
+
+        public CompletableFuture<T> future() {
+            return future;
+        }
+
+        @Override
+        public String toString() {
+            return "FutureEvent(key=" + key + ")";
+        }
+    }
+
+    @Test
+    public void testCreateAndClose() throws Exception {
+        CoordinatorEventProcessor eventProcessor = new MultiThreadedEventProcessor(
+            new LogContext(),
+            "event-processor-",
+            2
+        );
+        eventProcessor.close();
+    }
+
+    @Test
+    public void testEventsAreProcessed() throws Exception {
+        try (CoordinatorEventProcessor eventProcessor = new MultiThreadedEventProcessor(
+            new LogContext(),
+            "event-processor-",
+            2
+        )) {
+            AtomicInteger numEventsExecuted = new AtomicInteger(0);
+
+            List<FutureEvent<Integer>> events = Arrays.asList(
+                new FutureEvent<>(0, numEventsExecuted::incrementAndGet),
+                new FutureEvent<>(1, numEventsExecuted::incrementAndGet),
+                new FutureEvent<>(2, numEventsExecuted::incrementAndGet),
+                new FutureEvent<>(0, numEventsExecuted::incrementAndGet),
+                new FutureEvent<>(1, numEventsExecuted::incrementAndGet),
+                new FutureEvent<>(2, numEventsExecuted::incrementAndGet)
+            );
+
+            events.forEach(eventProcessor::enqueue);
+
+            CompletableFuture.allOf(events
+                .stream()
+                .map(FutureEvent::future)
+                .toArray(CompletableFuture[]::new)
+            ).get(10, TimeUnit.SECONDS);
+
+            events.forEach(event -> {
+                assertTrue(event.future.isDone());
+                assertFalse(event.future.isCompletedExceptionally());
+            });
+
+            assertEquals(events.size(), numEventsExecuted.get());
+        }
+    }
+
+    @Test
+    public void testEventsAreRejectedWhenClosed() throws Exception {
+        CoordinatorEventProcessor eventProcessor = new MultiThreadedEventProcessor(
+            new LogContext(),
+            "event-processor-",
+            2
+        );
+
+        eventProcessor.close();
+
+        assertThrows(RejectedExecutionException.class,
+            () -> eventProcessor.enqueue(new FutureEvent<>(0, () -> 0)));
+    }
+
+    @Test
+    public void testEventsAreDrainedWhenClosed() throws Exception {
+        try (MultiThreadedEventProcessor eventProcessor = new MultiThreadedEventProcessor(
+            new LogContext(),
+            "event-processor-",
+            1

Review Comment:
   Does this test require only a single thread in order to block the other events from being added? 



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.group.runtime;
+
+import org.apache.kafka.common.utils.LogContext;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import static org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS;
+import static org.apache.kafka.test.TestUtils.waitForCondition;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@Timeout(value = 60)
+public class MultiThreadedEventProcessorTest {
+
+    private static class FutureEvent<T> implements CoordinatorEvent {
+        private final int key;
+        private final CompletableFuture<T> future;
+        private final Supplier<T> supplier;
+
+        FutureEvent(
+            int key,
+            Supplier<T> supplier
+        ) {
+            this.key = key;
+            this.future = new CompletableFuture<>();
+            this.supplier = supplier;
+        }
+
+        @Override
+        public void run() {
+            future.complete(supplier.get());
+        }
+
+        @Override
+        public void complete(Throwable ex) {
+            future.completeExceptionally(ex);
+        }
+
+        @Override
+        public Integer key() {
+            return key;
+        }
+
+        public CompletableFuture<T> future() {
+            return future;
+        }
+
+        @Override
+        public String toString() {
+            return "FutureEvent(key=" + key + ")";
+        }
+    }
+
+    @Test
+    public void testCreateAndClose() throws Exception {
+        CoordinatorEventProcessor eventProcessor = new MultiThreadedEventProcessor(

Review Comment:
   Are there any conditions you want to check here? Or just ensure these operations don't throw errors?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org