philipnee commented on code in PR #12672:
URL: https://github.com/apache/kafka/pull/12672#discussion_r998532115


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/DefaultBackgroundThread.java:
##########
@@ -0,0 +1,226 @@
+/*
+ * 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;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent;
+import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent;
+import org.apache.kafka.clients.consumer.internals.events.NoopApplicationEvent;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.errors.InterruptException;
+import org.apache.kafka.common.errors.WakeupException;
+import org.apache.kafka.common.metrics.Metrics;
+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.util.Optional;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * 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.
+ *
+ * It holds a reference to the {@link SubscriptionState}, which is
+ * initialized by the polling thread.
+ */
+public class DefaultBackgroundThread extends KafkaThread {
+    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 ConsumerNetworkClient networkClient;
+    private final SubscriptionState subscriptions;
+    private final ConsumerMetadata metadata;
+    private final Metrics metrics;
+    private final ConsumerConfig config;
+
+    private String clientId;
+    private long retryBackoffMs;
+    private int heartbeatIntervalMs;
+    private boolean running;
+    private Optional<ApplicationEvent> inflightEvent = Optional.empty();
+    private final AtomicReference<Optional<RuntimeException>> exception =
+            new AtomicReference<>(Optional.empty());
+
+    public DefaultBackgroundThread(ConsumerConfig config,
+                                   LogContext logContext,
+                                   BlockingQueue<ApplicationEvent> 
applicationEventQueue,
+                                   BlockingQueue<BackgroundEvent> 
backgroundEventQueue,
+                                   SubscriptionState subscriptions,
+                                   ConsumerMetadata metadata,
+                                   ConsumerNetworkClient networkClient,
+                                   Metrics metrics) {
+        this(Time.SYSTEM,
+                config,
+                logContext,
+                applicationEventQueue,
+                backgroundEventQueue,
+                subscriptions,
+                metadata,
+                networkClient,
+                metrics);
+    }
+
+    public DefaultBackgroundThread(Time time,
+                                   ConsumerConfig config,
+                                   LogContext logContext,
+                                   BlockingQueue<ApplicationEvent> 
applicationEventQueue,
+                                   BlockingQueue<BackgroundEvent> 
backgroundEventQueue,
+                                   SubscriptionState subscriptions,
+                                   ConsumerMetadata metadata,
+                                   ConsumerNetworkClient networkClient,
+                                   Metrics metrics) {
+        super(BACKGROUND_THREAD_NAME, true);
+        try {
+            this.time = time;
+            this.log = logContext.logger(DefaultBackgroundThread.class);
+            this.applicationEventQueue = applicationEventQueue;
+            this.backgroundEventQueue = backgroundEventQueue;
+            this.config = config;
+            setConfig();
+            this.inflightEvent = Optional.empty();
+            // subscriptionState is initialized by the polling thread
+            this.subscriptions = subscriptions;
+            this.metadata = metadata;
+            this.networkClient = networkClient;
+            this.metrics = metrics;
+            this.running = true;
+        } catch (Exception e) {
+            // now propagate the exception
+            close();
+            throw new KafkaException("Failed to construct background 
processor", e);
+        }
+    }
+
+    private void setConfig() {
+        this.retryBackoffMs = 
this.config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
+        this.clientId = config.getString(CommonClientConfigs.CLIENT_ID_CONFIG);
+        this.heartbeatIntervalMs = 
config.getInt(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG);
+    }
+
+    @Override
+    public void run() {
+        try {
+            log.debug("{} started", getClass());
+            while (running) {
+                try {
+                    runOnce();
+                } catch (WakeupException | InterruptException e) {
+                    log.debug("Exception thrown, background thread won't " +
+                            "terminate", e);
+                    // swallow
+                }
+            }
+        } catch (Throwable t) {
+            log.error("The background thread failed due to unexpected error",
+                    t);
+            if (t instanceof RuntimeException)
+                this.exception.set(Optional.of((RuntimeException) t));

Review Comment:
   We could, there are other places in the code that uses RTE so trying to be 
consistent with the existing code.



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