chia7712 commented on code in PR #21261:
URL: https://github.com/apache/kafka/pull/21261#discussion_r2678865866


##########
server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java:
##########
@@ -207,4 +208,30 @@ private static String parseListenerName(String 
connectionString) {
         }
         return connectionString.substring(0, 
firstColon).toUpperCase(Locale.ROOT);
     }
+
+    /**
+     * Registers a component for dynamic reconfiguration notifications.
+     * <p>
+     * This method exists to support migration from kafka.server.KafkaConfig 
(Scala/core) to AbstractKafkaConfig (Java/server).
+     * When migrating code, replace KafkaConfig references with 
AbstractKafkaConfig. The default implementation is a no-op;
+     * KafkaConfig overrides this to manage the reconfigurable registry.
+     *
+     * @param reconfigurable the component to register for configuration 
updates
+     */
+    public void addReconfigurable(Reconfigurable reconfigurable) {

Review Comment:
   why we don't set it as abstract method?



##########
server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java:
##########
@@ -207,4 +208,30 @@ private static String parseListenerName(String 
connectionString) {
         }
         return connectionString.substring(0, 
firstColon).toUpperCase(Locale.ROOT);
     }
+
+    /**
+     * Registers a component for dynamic reconfiguration notifications.
+     * <p>
+     * This method exists to support migration from kafka.server.KafkaConfig 
(Scala/core) to AbstractKafkaConfig (Java/server).
+     * When migrating code, replace KafkaConfig references with 
AbstractKafkaConfig. The default implementation is a no-op;
+     * KafkaConfig overrides this to manage the reconfigurable registry.
+     *
+     * @param reconfigurable the component to register for configuration 
updates
+     */
+    public void addReconfigurable(Reconfigurable reconfigurable) {
+        // Default no-op. Overridden by KafkaConfig to support dynamic 
configuration.
+    }
+
+    /**
+     * Unregisters a component from dynamic reconfiguration notifications.
+     * <p>
+     * This method exists to support migration from kafka.server.KafkaConfig 
(Scala/core) to AbstractKafkaConfig (Java/server).
+     * When migrating code, replace KafkaConfig references with 
AbstractKafkaConfig. The default implementation is a no-op;
+     * KafkaConfig overrides this to remove components from the registry.
+     *
+     * @param reconfigurable the component to unregister
+     */
+    public void removeReconfigurable(Reconfigurable reconfigurable) {

Review Comment:
   ditto



##########
server/src/main/java/org/apache/kafka/server/NodeToControllerChannelManagerImpl.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.server;
+
+import org.apache.kafka.clients.ApiVersions;
+import org.apache.kafka.clients.KafkaClient;
+import org.apache.kafka.clients.ManualMetadataUpdater;
+import org.apache.kafka.clients.MetadataRecoveryStrategy;
+import org.apache.kafka.clients.NetworkClient;
+import org.apache.kafka.clients.NodeApiVersions;
+import org.apache.kafka.common.Reconfigurable;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.network.ChannelBuilder;
+import org.apache.kafka.common.network.ChannelBuilders;
+import org.apache.kafka.common.network.NetworkReceive;
+import org.apache.kafka.common.network.Selectable;
+import org.apache.kafka.common.network.Selector;
+import org.apache.kafka.common.requests.AbstractRequest;
+import org.apache.kafka.common.security.JaasContext;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.raft.KRaftConfigs;
+import org.apache.kafka.server.common.ControllerRequestCompletionHandler;
+import org.apache.kafka.server.common.NodeToControllerChannelManager;
+import org.apache.kafka.server.config.AbstractKafkaConfig;
+import org.apache.kafka.server.config.ReplicationConfigs;
+import org.apache.kafka.server.config.ServerConfigs;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Manages a communication channel from a node to the active KRaft controller.
+ * <p>
+ * Creates a network client with the appropriate security configuration and 
uses a background
+ * request thread to queue and send requests asynchronously. Supports dynamic 
reconfiguration
+ * of security settings when the controller listener configuration changes.
+ */
+public class NodeToControllerChannelManagerImpl implements 
NodeToControllerChannelManager {
+    private static final Logger log = 
LoggerFactory.getLogger(NodeToControllerChannelManagerImpl.class);
+    private final ControllerNodeProvider controllerNodeProvider;
+    private final Time time;
+    private final Metrics metrics;
+    private final AbstractKafkaConfig config;
+    private final String channelName;
+    private final String threadNamePrefix;
+    private final Long retryTimeoutMs;
+
+    private final LogContext logContext;
+    private final ManualMetadataUpdater manualMetadataUpdater = new 
ManualMetadataUpdater();
+    private final ApiVersions apiVersions = new ApiVersions();
+    private final NodeToControllerRequestThread requestThread;
+
+    @SuppressWarnings("this-escape")

Review Comment:
   Would you mind inlining `newRequestThread` to get rid of this warning?



##########
server/src/main/java/org/apache/kafka/server/ControllerNodeProvider.java:
##########
@@ -0,0 +1,25 @@
+/*
+ * 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.server;
+
+/**
+ * Discovers the active controller node and provides connection details for 
communicating with it.
+ */
+public interface ControllerNodeProvider {

Review Comment:
   Could we use `Supplier<ControllerInformation>` instead?



##########
server/src/main/java/org/apache/kafka/server/NodeToControllerRequestThread.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.server;
+
+import org.apache.kafka.clients.ClientResponse;
+import org.apache.kafka.clients.KafkaClient;
+import org.apache.kafka.clients.ManualMetadataUpdater;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.config.AbstractConfig;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.server.config.ReplicationConfigs;
+import org.apache.kafka.server.util.InterBrokerSendThread;
+import org.apache.kafka.server.util.RequestAndCompletionHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Background thread that manages to send requests to the active controller.
+ * <p>
+ * Maintains a queue of pending requests and handles automatic retries on 
failures,
+ * controller failover, and timeout detection. Requests are re-queued when the 
controller
+ * changes or connections are lost.
+ */
+public class NodeToControllerRequestThread extends InterBrokerSendThread {
+    private static final Logger log = 
LoggerFactory.getLogger(NodeToControllerRequestThread.class);
+
+    private final LinkedBlockingDeque<NodeToControllerQueueItem> requestQueue 
= new LinkedBlockingDeque<>();
+    private final AtomicReference<Node> activeController = new 
AtomicReference<>(null);
+
+
+    private final Time time;
+    private final long retryTimeoutMs;
+    private final ControllerNodeProvider controllerNodeProvider;
+    private final ManualMetadataUpdater metadataUpdater;
+
+    // Used for testing
+    volatile boolean started = false;
+    public void setStarted(boolean started) {
+        this.started = started;
+    }
+
+    public NodeToControllerRequestThread(KafkaClient initialNetworkClient,
+                                         ManualMetadataUpdater metadataUpdater,
+                                         ControllerNodeProvider 
controllerNodeProvider,
+                                         AbstractConfig config,
+                                         Time time,
+                                         String threadName,
+                                         Long retryTimeoutMs) {
+        super(threadName, initialNetworkClient, Math.min(Integer.MAX_VALUE, 
(int) 
Math.min(config.getInt(ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG), 
retryTimeoutMs)), time, false);
+        this.time = time;
+        this.controllerNodeProvider = controllerNodeProvider;
+        this.metadataUpdater = metadataUpdater;
+        this.retryTimeoutMs = retryTimeoutMs;
+    }
+
+    public Optional<Node> activeControllerAddress() {
+        return Optional.ofNullable(activeController.get());
+    }
+
+    private void updateControllerAddress(Node newActiveController) {
+        activeController.set(newActiveController);
+    }
+
+    public void enqueue(NodeToControllerQueueItem request) {
+        if (!started) {
+            throw new IllegalStateException("Cannot enqueue a request if the 
request thread is not running");
+        }
+        requestQueue.add(request);
+        if (activeControllerAddress().isPresent()) {
+            wakeup();
+        }
+    }
+
+    public int queueSize() {
+        return requestQueue.size();
+    }
+
+    @Override
+    public Collection<RequestAndCompletionHandler> generateRequests() {
+        final long currentTimeMs = time.milliseconds();
+        final Iterator<NodeToControllerQueueItem> requestIter = 
requestQueue.iterator();
+        while (requestIter.hasNext()) {
+            var request = requestIter.next();
+            if (currentTimeMs - request.createdTimeMs() >= retryTimeoutMs) {
+                requestIter.remove();
+                request.callback().onTimeout();
+            } else {
+                Optional<Node> controllerAddress = activeControllerAddress();
+                if (controllerAddress.isPresent()) {
+                    requestIter.remove();
+                    return Collections.singletonList(new 
RequestAndCompletionHandler(
+                            time.milliseconds(),
+                            controllerAddress.get(),
+                            request.request(),
+                            response -> handleResponse(request, response)
+                    ));
+                }
+            }
+        }
+
+        return Collections.emptyList();
+    }
+
+    void handleResponse(NodeToControllerQueueItem queueItem, ClientResponse 
response) {
+        log.debug("Request {} received {}", queueItem.request(), response);
+        if (response.authenticationException() != null) {
+            log.error("Request {} failed due to authentication error with 
controller. Disconnecting the " +
+                            "connection to the stale controller {}",
+                    queueItem.request(), 
activeControllerAddress().map(Node::idString).orElse("null"),
+                    response.authenticationException()
+            );
+            maybeDisconnectAndUpdateController();
+            queueItem.callback().onComplete(response);
+        } else if (response.versionMismatch() != null) {
+            log.error("Request {} failed due to unsupported version error", 
queueItem.request(),
+                    response.versionMismatch());
+            queueItem.callback().onComplete(response);
+        } else if (response.wasDisconnected()) {
+            updateControllerAddress(null);
+            try {
+                requestQueue.putFirst(queueItem);

Review Comment:
   We can substitute `addFirst` for `putFirst`, since `requestQueue` is an 
unbounded `LinkedBlockingDeque`



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to