cmccabe commented on code in PR #14306:
URL: https://github.com/apache/kafka/pull/14306#discussion_r1312372436


##########
core/src/main/scala/kafka/server/ControllerRegistrationManager.scala:
##########
@@ -0,0 +1,307 @@
+/**
+ * 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 kafka.server
+
+import java.util
+import java.util.concurrent.TimeUnit.MILLISECONDS
+import kafka.utils.Logging
+import org.apache.kafka.clients.ClientResponse
+import org.apache.kafka.common.Uuid
+import 
org.apache.kafka.common.message.ControllerRegistrationRequestData.ListenerCollection
+import org.apache.kafka.common.message.ControllerRegistrationRequestData
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{ControllerRegistrationRequest, 
ControllerRegistrationResponse}
+import org.apache.kafka.metadata.VersionRange
+import org.apache.kafka.common.utils.{ExponentialBackoff, LogContext, Time}
+import org.apache.kafka.image.loader.LoaderManifest
+import org.apache.kafka.image.{MetadataDelta, MetadataImage}
+import org.apache.kafka.image.publisher.MetadataPublisher
+import org.apache.kafka.queue.EventQueue.DeadlineFunction
+import org.apache.kafka.queue.{EventQueue, KafkaEventQueue}
+import org.apache.kafka.server.common.MetadataVersion
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * The broker lifecycle manager owns the broker state.
+ *
+ * Its inputs are messages passed in from other parts of the broker and from 
the
+ * controller: requests to start up, or shut down, for example. Its output are 
the broker
+ * state and various futures that can be used to wait for broker state 
transitions to
+ * occur.
+ *
+ * The lifecycle manager handles registering the broker with the controller, 
as described
+ * in KIP-631. After registration is complete, it handles sending periodic 
broker
+ * heartbeats and processing the responses.
+ *
+ * This code uses an event queue paradigm. Modifications get translated into 
events, which
+ * are placed on the queue to be processed sequentially. As described in the 
JavaDoc for
+ * each variable, most mutable state can be accessed only from that event 
queue thread.
+ * In some cases we expose a volatile variable which can be read from any 
thread, but only
+ * written from the event queue thread.
+ */
+class ControllerRegistrationManager(
+  val config: KafkaConfig,
+  val clusterId: String,
+  val time: Time,
+  val threadNamePrefix: String,
+  val supportedFeatures: util.Map[String, VersionRange],
+  val incarnationId: Uuid,
+  val resendExponentialBackoff: ExponentialBackoff = new 
ExponentialBackoff(100, 2, 120000L, 0.02)
+) extends Logging with MetadataPublisher {
+  override def name(): String = "ControllerRegistrationManager"
+
+  val nodeId: Int = config.nodeId
+
+  private def logPrefix(): String = {
+    val builder = new StringBuilder("[ControllerRegistrationManager")
+    builder.append(" id=").append(config.nodeId)
+    builder.append(" incarnation=").append(incarnationId)
+    builder.append("] ")
+    builder.toString()
+  }
+
+  val logContext = new LogContext(logPrefix())
+
+  this.logIdent = logContext.logPrefix()
+
+  val listenerCollection = {
+    val collection = new ListenerCollection()
+    config.controllerListeners.foreach(endPoint => {
+      collection.add(new ControllerRegistrationRequestData.Listener().
+        setHost(endPoint.host).
+        setName(endPoint.listenerName.value()).
+        setPort(endPoint.port).
+        setSecurityProtocol(endPoint.securityProtocol.id))
+    })
+    collection
+  }
+
+  /**
+   * The number of RPCs that we are waiting for. Only read or written from the 
event queue thread.
+   */
+  var pendingRpcs = 0L
+
+  /**
+   * The number of RPCs that we successfully sent.
+   * Only read or written from the event queue thread.
+   */
+  var successfulRpcs = 0L
+
+  /**
+   * The number of RPCs that we failed to send, or got back a failure response 
for. This is
+   * cleared after a success. Only read or written from the event queue thread.
+   */
+  var failedRpcs = 0L
+
+  /**
+   * The current metadata version that is in effect. Only read or written from 
the event queue thread.
+   */
+  private var metadataVersion: MetadataVersion = 
MetadataVersion.MINIMUM_KRAFT_VERSION
+
+  /**
+   * True if we're registered. Only read or written from the event queue 
thread.
+   */
+  var registered: Boolean = false
+
+  /**
+   * The channel manager, or null if this manager has not been started yet.  
This variable
+   * can only be read or written from the event queue thread.
+   */
+  private var _channelManager: BrokerToControllerChannelManager = _
+
+  /**
+   * The event queue.
+   */
+  private[server] val eventQueue = new KafkaEventQueue(time,
+    logContext,
+    threadNamePrefix + "registration-manager-",
+    new ShutdownEvent())
+
+  private class ShutdownEvent extends EventQueue.Event {
+    override def run(): Unit = {
+      try {
+        info(s"shutting down.")
+        if (_channelManager != null) {
+          _channelManager.shutdown()
+          _channelManager = null
+        }
+      } catch {
+        case t: Throwable => error("ControllerRegistrationManager.stop error", 
t)
+      }
+    }
+  }
+
+  /**
+   * Start the BrokerLifecycleManager.
+   *
+   * @param channelManager                The brokerToControllerChannelManager 
to use.
+   */
+  def start(channelManager: BrokerToControllerChannelManager): Unit = {
+    eventQueue.append(() => {
+      try {
+        info(s"initialized channel manager.")
+        _channelManager = channelManager
+        maybeSendControllerRegistration()
+      } catch {
+        case t: Throwable => error("start error", t)
+      }
+    })
+  }
+
+  /**
+   * Start shutting down the BrokerLifecycleManager, but do not block.
+   */
+  def beginShutdown(): Unit = {
+    eventQueue.beginShutdown("beginShutdown");
+  }
+
+  /**
+   * Shut down the BrokerLifecycleManager and block until all threads are 
joined.
+   */
+  override def close(): Unit = {
+    beginShutdown()
+    eventQueue.close()
+  }
+
+  override def onMetadataUpdate(
+    delta: MetadataDelta,
+    newImage: MetadataImage,
+    manifest: LoaderManifest
+  ): Unit = {
+    if (delta.featuresDelta() != null ||
+        (delta.clusterDelta() != null && 
delta.clusterDelta().changedControllers().containsKey(nodeId))) {

Review Comment:
   yes, it cares only about its own registration 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: [email protected]

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

Reply via email to