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


##########
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
-    private final ApplicationEventProcessor applicationEventProcessor;
-    private final NetworkClientDelegate networkClientDelegate;
-    private final ErrorEventHandler errorEventHandler;
-    private final GroupState groupState;
-    private boolean running;
-
-    private final RequestManagers requestManagers;
-
-    // Visible for testing
-    DefaultBackgroundThread(final Time time,
-                            final ConsumerConfig config,
-                            final LogContext logContext,
-                            final BlockingQueue<ApplicationEvent> 
applicationEventQueue,
-                            final BlockingQueue<BackgroundEvent> 
backgroundEventQueue,
-                            final ErrorEventHandler errorEventHandler,
-                            final ApplicationEventProcessor processor,
-                            final ConsumerMetadata metadata,
-                            final NetworkClientDelegate networkClient,
-                            final GroupState groupState,
-                            final CoordinatorRequestManager coordinatorManager,
-                            final CommitRequestManager commitRequestManager,
-                            final OffsetsRequestManager offsetsRequestManager) 
{
+    private ApplicationEventProcessor applicationEventProcessor;
+    private NetworkClientDelegate networkClientDelegate;
+    private RequestManagers requestManagers;
+    private volatile boolean running;
+    private final IdempotentCloser closer = new IdempotentCloser();
+
+    public DefaultBackgroundThread(Time time,
+                                   LogContext logContext,
+                                   BlockingQueue<ApplicationEvent> 
applicationEventQueue,
+                                   Supplier<ApplicationEventProcessor> 
applicationEventProcessorSupplier,
+                                   Supplier<NetworkClientDelegate> 
networkClientDelegateSupplier,
+                                   Supplier<RequestManagers> 
requestManagersSupplier) {
         super(BACKGROUND_THREAD_NAME, true);
         this.time = time;
-        this.running = true;
         this.log = logContext.logger(getClass());
         this.applicationEventQueue = applicationEventQueue;
-        this.backgroundEventQueue = backgroundEventQueue;
-        this.applicationEventProcessor = processor;
-        this.config = config;
-        this.metadata = metadata;
-        this.networkClientDelegate = networkClient;
-        this.errorEventHandler = errorEventHandler;
-        this.groupState = groupState;
-
-        this.requestManagers = new RequestManagers(
-                offsetsRequestManager,
-                Optional.ofNullable(coordinatorManager),
-                Optional.ofNullable(commitRequestManager));
-    }
-
-    public DefaultBackgroundThread(final Time time,
-                                   final ConsumerConfig config,
-                                   final GroupRebalanceConfig rebalanceConfig,
-                                   final LogContext logContext,
-                                   final BlockingQueue<ApplicationEvent> 
applicationEventQueue,
-                                   final BlockingQueue<BackgroundEvent> 
backgroundEventQueue,
-                                   final ConsumerMetadata metadata,
-                                   final SubscriptionState subscriptionState,
-                                   final ApiVersions apiVersions,
-                                   final Metrics metrics,
-                                   final Sensor fetcherThrottleTimeSensor) {
-        super(BACKGROUND_THREAD_NAME, true);
-        requireNonNull(config);
-        requireNonNull(rebalanceConfig);
-        requireNonNull(logContext);
-        requireNonNull(applicationEventQueue);
-        requireNonNull(backgroundEventQueue);
-        requireNonNull(metadata);
-        requireNonNull(subscriptionState);
-        try {
-            this.time = time;
-            this.log = logContext.logger(getClass());
-            this.applicationEventQueue = applicationEventQueue;
-            this.backgroundEventQueue = backgroundEventQueue;
-            this.config = config;
-            this.metadata = metadata;
-            final NetworkClient networkClient = 
ClientUtils.createNetworkClient(config,
-                    metrics,
-                    CONSUMER_METRIC_GROUP_PREFIX,
-                    logContext,
-                    apiVersions,
-                    time,
-                    CONSUMER_MAX_INFLIGHT_REQUESTS_PER_CONNECTION,
-                    metadata,
-                    fetcherThrottleTimeSensor);
-            this.networkClientDelegate = new NetworkClientDelegate(
-                    this.time,
-                    this.config,
-                    logContext,
-                    networkClient);
-            this.running = true;
-            this.errorEventHandler = new 
ErrorEventHandler(this.backgroundEventQueue);
-            this.groupState = new GroupState(rebalanceConfig);
-            long retryBackoffMs = 
config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
-            long retryBackoffMaxMs = 
config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG);
-            final int requestTimeoutMs = 
config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG);
-
-            OffsetsRequestManager offsetsRequestManager =
-                    new OffsetsRequestManager(
-                            subscriptionState,
-                            metadata,
-                            configuredIsolationLevel(config),
-                            time,
-                            retryBackoffMs,
-                            requestTimeoutMs,
-                            apiVersions,
-                            networkClientDelegate,
-                            logContext);
-            CoordinatorRequestManager coordinatorRequestManager = null;
-            CommitRequestManager commitRequestManager = null;
-
-            if (groupState.groupId != null) {
-                coordinatorRequestManager = new CoordinatorRequestManager(
-                        this.time,
-                        logContext,
-                        retryBackoffMs,
-                        retryBackoffMaxMs,
-                        this.errorEventHandler,
-                        groupState.groupId);
-                commitRequestManager = new CommitRequestManager(
-                        this.time,
-                        logContext,
-                        subscriptionState,
-                        config,
-                        coordinatorRequestManager,
-                        groupState);
-            }
-
-            this.requestManagers = new RequestManagers(
-                    offsetsRequestManager,
-                    Optional.ofNullable(coordinatorRequestManager),
-                    Optional.ofNullable(commitRequestManager));
-
-            this.applicationEventProcessor = new ApplicationEventProcessor(
-                    backgroundEventQueue,
-                    requestManagers,
-                    metadata);
-
-        } catch (final Exception e) {
-            close();
-            throw new KafkaException("Failed to construct background 
processor", e.getCause());
-        }
+        this.applicationEventProcessorSupplier = 
applicationEventProcessorSupplier;
+        this.networkClientDelegateSupplier = networkClientDelegateSupplier;
+        this.requestManagersSupplier = requestManagersSupplier;
     }
 
     @Override
     public void run() {
+        closer.assertOpen("Consumer background thread is already closed");

Review Comment:
   I find this name combination closer+assertOpen kind of confusing. I even 
think it would it be easier to follow if we just had the explicit check 
`if(isClose()) throw` 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-unsubscr...@kafka.apache.org

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

Reply via email to