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


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventProcessor.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.clients.consumer.internals.events;
+
+import org.apache.kafka.common.utils.LogContext;
+import org.slf4j.Logger;
+
+import java.util.LinkedList;
+import java.util.Objects;
+import java.util.concurrent.BlockingQueue;
+
+public class BackgroundEventProcessor {
+
+    private final Logger log;
+    private final BlockingQueue<BackgroundEvent> backgroundEventQueue;
+
+    public BackgroundEventProcessor(final LogContext logContext,
+                                    final BlockingQueue<BackgroundEvent> 
backgroundEventQueue) {
+        this.log = logContext.logger(BackgroundEventProcessor.class);
+        this.backgroundEventQueue = backgroundEventQueue;
+    }
+
+    /**
+     * Drains all available {@link BackgroundEvent}s, and then processes them 
in order. If any
+     * errors are thrown as a result of a {@link ErrorBackgroundEvent} or an 
error occurs while processing
+     * another type of {@link BackgroundEvent}, only the <em>first</em> 
exception will be thrown, all
+     * subsequent errors will simply be logged at <code>WARN</code> level.
+     *
+     * @throws RuntimeException or subclass
+     */
+    public void process() {
+        LinkedList<BackgroundEvent> events = new LinkedList<>();
+        backgroundEventQueue.drainTo(events);
+
+        RuntimeException first = null;
+        int errorCount = 0;
+
+        for (BackgroundEvent event : events) {
+            log.debug("Consuming background event: {}", event);
+
+            try {
+                process(event);
+            } catch (RuntimeException e) {
+                errorCount++;
+
+                if (first == null) {
+                    first = e;
+                    log.warn("Error #{} from background thread (will be logged 
and thrown): {}", errorCount, e.getMessage(), e);

Review Comment:
   Removed check to avoid.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/DefaultBackgroundThread.java:
##########
@@ -16,197 +16,78 @@
  */
 package org.apache.kafka.clients.consumer.internals;
 
-import org.apache.kafka.clients.ApiVersions;
-import org.apache.kafka.clients.ClientUtils;
-import org.apache.kafka.clients.GroupRebalanceConfig;
-import org.apache.kafka.clients.NetworkClient;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent;
 import 
org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor;
-import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent;
+import 
org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent;
 import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.errors.WakeupException;
-import org.apache.kafka.common.metrics.Metrics;
-import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.internals.IdempotentCloser;
 import org.apache.kafka.common.utils.KafkaThread;
 import org.apache.kafka.common.utils.LogContext;
 import org.apache.kafka.common.utils.Time;
 import org.apache.kafka.common.utils.Utils;
 import org.slf4j.Logger;
 
+import java.io.Closeable;
+import java.util.ArrayList;
 import java.util.LinkedList;
+import java.util.List;
 import java.util.Objects;
 import java.util.Optional;
 import java.util.concurrent.BlockingQueue;
-
-import static java.util.Objects.requireNonNull;
-import static 
org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_MAX_INFLIGHT_REQUESTS_PER_CONNECTION;
-import static 
org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX;
-import static 
org.apache.kafka.clients.consumer.internals.ConsumerUtils.configuredIsolationLevel;
+import java.util.function.Supplier;
 
 /**
  * Background thread runnable that consumes {@code ApplicationEvent} and
  * produces {@code BackgroundEvent}. It uses an event loop to consume and
  * produce events, and poll the network client to handle network IO.
- * <p/>
+ * <p>
  * It holds a reference to the {@link SubscriptionState}, which is
  * initialized by the polling thread.
- * <p/>
- * For processing application events that have been submitted to the
- * {@link #applicationEventQueue}, this relies on an {@link 
ApplicationEventProcessor}. Processing includes generating requests and
- * handling responses with the appropriate {@link RequestManager}. The network 
operations for
- * actually sending the requests is delegated to the {@link 
NetworkClientDelegate}
- * </li>
  */
-public class DefaultBackgroundThread extends KafkaThread {
+public class DefaultBackgroundThread extends KafkaThread implements Closeable {
+
     private static final long MAX_POLL_TIMEOUT_MS = 5000;
     private static final String BACKGROUND_THREAD_NAME = 
"consumer_background_thread";
     private final Time time;
     private final Logger log;
     private final BlockingQueue<ApplicationEvent> applicationEventQueue;
-    private final BlockingQueue<BackgroundEvent> backgroundEventQueue;
-    private final ConsumerMetadata metadata;
-    private final ConsumerConfig config;
+    private final Supplier<ApplicationEventProcessor> 
applicationEventProcessorSupplier;
+    private final Supplier<NetworkClientDelegate> 
networkClientDelegateSupplier;
+    private final Supplier<RequestManagers> requestManagersSupplier;
     // empty if groupId is null

Review Comment:
   Done.



-- 
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-unsubscr...@kafka.apache.org

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

Reply via email to