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

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

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


##########
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 {

Review Comment:
   does EventAccumulatorTest#testKeyConcurrentProcessingAndOrdering() actually test concurrent ordering? i'm wondering if we can add one here or in EventAccumulatorTest.



##########
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:
   do we need the lock in enqueue/shutdown?
   
   > prevent the queue from never emptying
   
   once we set shuttingDown=true in `beginShutdown()`, shouldn't the other threads learn to `drainEvents()` without locks?



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