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

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

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.apache.kafka.server.util.ShutdownableThread;
+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 ShutdownableThread {
+        EventProcessorThread(
+            String name
+        ) {
+            super(name, false);
+        }
+
+        @Override
+        public void doWork() {
+            while (!shuttingDown) {

Review Comment:
   Should the `while` loop condition also include checking some of the methods in `ShutdownableThread`?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.apache.kafka.server.util.ShutdownableThread;
+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 ShutdownableThread {
+        EventProcessorThread(
+            String name
+        ) {
+            super(name, false);
+        }
+
+        @Override
+        public void doWork() {
+            while (!shuttingDown) {
+                CoordinatorEvent event = accumulator.poll();
+                if (event == null) continue;
+
+                try {
+                    log.debug("Executing event " + event);
+                    event.run();
+                } catch (Throwable t) {
+                    log.error("Failed to run event " + event + " due to: " + t, t);

Review Comment:
   nit: Any reason to not use the logging placeholders (`{}`)? 



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorEvent.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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;
+
+/**
+ * The base event type used by all events processed in the
+ * coordinator runtime.
+ */
+public interface CoordinatorEvent extends EventAccumulator.Event<Integer> {
+    /**
+     * Runs the event.
+     */
+    void run();

Review Comment:
   What are the benefits/drawbacks from having this interface extend `Runnable` instead of adding `run` here?



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