You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2022/11/10 18:12:19 UTC

[GitHub] [kafka] hachikuji commented on a diff in pull request #12837: MINOR: extract jointly owned parts of BrokerServer and ControllerServer

hachikuji commented on code in PR #12837:
URL: https://github.com/apache/kafka/pull/12837#discussion_r1019455482


##########
core/src/main/scala/kafka/server/JointServer.scala:
##########
@@ -0,0 +1,237 @@
+/**
+ * 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 kafka.raft.KafkaRaftManager
+import kafka.server.KafkaRaftServer.{BrokerRole, ControllerRole}
+import kafka.server.metadata.BrokerServerMetrics
+import kafka.utils.{CoreUtils, Logging}
+import org.apache.kafka.common.metrics.Metrics
+import org.apache.kafka.common.utils.{LogContext, Time}
+import org.apache.kafka.controller.QuorumControllerMetrics
+import org.apache.kafka.metadata.MetadataRecordSerde
+import org.apache.kafka.raft.RaftConfig.AddressSpec
+import org.apache.kafka.server.common.ApiMessageAndVersion
+import org.apache.kafka.server.fault.{FaultHandler, LoggingFaultHandler, ProcessExitingFaultHandler}
+import org.apache.kafka.server.metrics.KafkaYammerMetrics
+
+import java.util
+import java.util.concurrent.CompletableFuture
+
+
+/**
+ * Creates a fault handler.
+ */
+trait FaultHandlerFactory {
+  def build(
+     name: String,
+     fatal: Boolean,
+     action: Runnable
+  ): FaultHandler
+}
+
+/**
+ * The standard FaultHandlerFactory which is used when we're not in a junit test.
+ */
+class StandardFaultHandlerFactory extends FaultHandlerFactory {
+  override def build(
+    name: String,
+    fatal: Boolean,
+    action: Runnable
+  ): FaultHandler = {
+    if (fatal) {
+      new ProcessExitingFaultHandler(action)
+    } else {
+      new LoggingFaultHandler(name, action)
+    }
+  }
+}
+
+/**
+ * The JointServer manages the components which are shared between the BrokerServer and
+ * ControllerServer. These shared components include the Raft manager, snapshot generator,
+ * and metadata loader. A KRaft server running in combined mode as both a broker and a controller
+ * will still contain only a single JointServer instance.
+ *
+ * The JointServer will be started as soon as either the broker or the controller needs it,
+ * via the appropriate function (startForBroker or startForController). Similarly, it will be
+ * stopped as soon as neither the broker nor the controller need it, via stopForBroker or
+ * stopForController. One way of thinking about this is that both the broker and the controller
+ * could hold a "reference" to this class, and we don't truly stop it until both have dropped
+ * their reference. We opted to use two booleans here rather than a reference count in order to
+ * make debugging easier and reduce the chance of resource leaks.
+ */
+class JointServer(
+  val config: KafkaConfig,
+  val metaProps: MetaProperties,
+  val time: Time,
+  val metrics: Metrics,
+  val threadNamePrefix: Option[String],
+  val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]],
+  val faultHandlerFactory: FaultHandlerFactory
+) extends Logging {
+  private val logContext: LogContext = new LogContext(s"[JointServer id=${config.nodeId}] ")
+  this.logIdent = logContext.logPrefix
+
+  var usedByBroker: Boolean = false
+  var usedByController: Boolean = false
+  var raftManager: KafkaRaftManager[ApiMessageAndVersion] = _
+  var brokerMetrics: BrokerServerMetrics = _
+  var controllerMetrics: QuorumControllerMetrics = _
+
+  def isUsed(): Boolean = synchronized {
+    usedByController || usedByBroker
+  }
+
+  /**
+   * The start function called by the broker.
+   */
+  def startForBroker(): Unit = synchronized {
+    if (!isUsed()) {
+      start()
+    }
+    usedByBroker = true
+  }
+
+  /**
+   * The start function called by the controller.
+   */
+  def startForController(): Unit = synchronized {
+    if (!isUsed()) {
+      start()
+    }
+    usedByController = true
+  }
+
+  /**
+   * The stop function called by the broker.
+   */
+  def stopForBroker(): Unit = synchronized {
+    if (usedByBroker) {
+      usedByBroker = false
+      if (!isUsed()) stop()
+    }
+  }
+
+  /**
+   * The stop function called by the controller.
+   */
+  def stopForController(): Unit = synchronized {
+    if (usedByController) {
+      usedByController = false
+      if (!isUsed()) stop()
+    }
+  }
+
+  /**
+   * The fault handler to use when metadata loading fails.
+   */
+  def metadataLoaderFaultHandler: FaultHandler = faultHandlerFactory.build("metadata loading",
+    config.processRoles.contains(ControllerRole),

Review Comment:
   nit: it's a little clearer if you provide the argument names, at least for the `fatal` arg. Same for other factories below.
   ```scala
   fatal = config.processRoles.contains(ControllerRole)
   ```



##########
core/src/main/scala/kafka/server/KafkaRaftServer.scala:
##########
@@ -69,95 +64,56 @@ class KafkaRaftServer(
   private val controllerQuorumVotersFuture = CompletableFuture.completedFuture(
     RaftConfig.parseVoterConnections(config.quorumVoters))
 
-  private val raftManager = new KafkaRaftManager[ApiMessageAndVersion](
-    metaProps,
+  private val jointServer = new JointServer(
     config,
-    new MetadataRecordSerde,
-    KafkaRaftServer.MetadataPartition,
-    KafkaRaftServer.MetadataTopicId,
+    metaProps,
     time,
     metrics,
     threadNamePrefix,
-    controllerQuorumVotersFuture
+    controllerQuorumVotersFuture,
+    new StandardFaultHandlerFactory(),
   )
 
   private val broker: Option[BrokerServer] = if (config.processRoles.contains(BrokerRole)) {
-    val brokerMetrics = BrokerServerMetrics(metrics)
-    val fatalFaultHandler = new ProcessExitingFaultHandler()
-    val metadataLoadingFaultHandler = new LoggingFaultHandler("metadata loading",
-        () => brokerMetrics.metadataLoadErrorCount.getAndIncrement())
-    val metadataApplyingFaultHandler = new LoggingFaultHandler("metadata application",
-      () => brokerMetrics.metadataApplyErrorCount.getAndIncrement())
     Some(new BrokerServer(
-      config,
-      metaProps,
-      raftManager,
-      time,
-      metrics,
-      brokerMetrics,
-      threadNamePrefix,
-      offlineDirs,
-      controllerQuorumVotersFuture,
-      fatalFaultHandler,
-      metadataLoadingFaultHandler,
-      metadataApplyingFaultHandler
+      jointServer,
+      offlineDirs
     ))
   } else {
     None
   }
 
   private val controller: Option[ControllerServer] = if (config.processRoles.contains(ControllerRole)) {
-    val controllerMetrics = new QuorumControllerMetrics(KafkaYammerMetrics.defaultRegistry(), time)
-    val metadataFaultHandler = new LoggingFaultHandler("controller metadata",
-      () => controllerMetrics.incrementMetadataErrorCount())
-    val fatalFaultHandler = new ProcessExitingFaultHandler()
     Some(new ControllerServer(
-      metaProps,
-      config,
-      raftManager,
-      time,
-      metrics,
-      controllerMetrics,
-      threadNamePrefix,
-      controllerQuorumVotersFuture,
+      jointServer,
       KafkaRaftServer.configSchema,
-      raftManager.apiVersions,
       bootstrapMetadata,
-      metadataFaultHandler,
-      fatalFaultHandler
     ))
   } else {
     None
   }
 
   override def startup(): Unit = {
     Mx4jLoader.maybeLoad()
-    // Note that we startup `RaftManager` first so that the controller and broker
-    // can register listeners during initialization.
-    raftManager.startup()
     controller.foreach(_.startup())
     broker.foreach(_.startup())
     AppInfoParser.registerAppInfo(Server.MetricsPrefix, config.brokerId.toString, metrics, time.milliseconds())
     info(KafkaBroker.STARTED_MESSAGE)
   }
 
   override def shutdown(): Unit = {
+    // In combined mode, we want to shut down the broker first, since it may take longer to complete.

Review Comment:
   We also need the controller to be available in order to process controlled shutdown.



##########
core/src/test/java/kafka/testkit/KafkaClusterTestKit.java:
##########
@@ -159,53 +218,30 @@ public KafkaClusterTestKit build() throws Exception {
                 executorService = Executors.newFixedThreadPool(numOfExecutorThreads,
                     ThreadUtils.createThreadFactory("KafkaClusterTestKit%d", false));
                 for (ControllerNode node : nodes.controllerNodes().values()) {
-                    Map<String, String> props = new HashMap<>(configProps);
-                    props.put(KafkaConfig$.MODULE$.ProcessRolesProp(), roles(node.id()));
-                    props.put(KafkaConfig$.MODULE$.NodeIdProp(),
-                        Integer.toString(node.id()));
-                    props.put(KafkaConfig$.MODULE$.MetadataLogDirProp(),
-                        node.metadataDirectory());
-                    props.put(KafkaConfig$.MODULE$.ListenerSecurityProtocolMapProp(),
-                        "EXTERNAL:PLAINTEXT,CONTROLLER:PLAINTEXT");
-                    props.put(KafkaConfig$.MODULE$.ListenersProp(), listeners(node.id()));
-                    props.put(KafkaConfig$.MODULE$.InterBrokerListenerNameProp(),
-                        nodes.interBrokerListenerName().value());
-                    props.put(KafkaConfig$.MODULE$.ControllerListenerNamesProp(),
-                        "CONTROLLER");
-                    // Note: we can't accurately set controller.quorum.voters yet, since we don't
-                    // yet know what ports each controller will pick.  Set it to a dummy string \
-                    // for now as a placeholder.
-                    props.put(RaftConfig.QUORUM_VOTERS_CONFIG, uninitializedQuorumVotersString);
-
-                    // reduce log cleaner offset map memory usage
-                    props.put(KafkaConfig$.MODULE$.LogCleanerDedupeBufferSizeProp(), "2097152");
-
                     setupNodeDirectories(baseDirectory, node.metadataDirectory(), Collections.emptyList());
-                    KafkaConfig config = new KafkaConfig(props, false, Option.empty());
-
-                    String threadNamePrefix = String.format("controller%d_", node.id());
-                    MetaProperties metaProperties = MetaProperties.apply(nodes.clusterId().toString(), node.id());
-                    TopicPartition metadataPartition = new TopicPartition(KafkaRaftServer.MetadataTopic(), 0);
                     BootstrapMetadata bootstrapMetadata = BootstrapMetadata.
                         fromVersion(nodes.bootstrapMetadataVersion(), "testkit");
-                    KafkaRaftManager<ApiMessageAndVersion> raftManager = new KafkaRaftManager<>(
-                        metaProperties, config, new MetadataRecordSerde(), metadataPartition, KafkaRaftServer.MetadataTopicId(),
-                        Time.SYSTEM, new Metrics(), Option.apply(threadNamePrefix), connectFutureManager.future);
-                    ControllerServer controller = new ControllerServer(
-                        nodes.controllerProperties(node.id()),
-                        config,
-                        raftManager,
-                        Time.SYSTEM,
-                        new Metrics(),
-                        new MockControllerMetrics(),
-                        Option.apply(threadNamePrefix),
-                        connectFutureManager.future,
-                        KafkaRaftServer.configSchema(),
-                        raftManager.apiVersions(),
-                        bootstrapMetadata,
-                        metadataFaultHandler,
-                        fatalFaultHandler
-                    );
+                    String threadNamePrefix = (nodes.brokerNodes().containsKey(node.id())) ?
+                            String.format("colocated%d", node.id()) :
+                            String.format("controller%d", node.id());
+                    JointServer jointServer = new JointServer(createNodeConfig(node),

Review Comment:
   We end up doing a bunch of stuff that `KafkaRaftServer` is already doing. Seems like it would be simpler to use `KafkaRaftServer` directly and get rid of all this logic to manage the lower-level components. That would also get us more consistent shutdown behavior.



##########
core/src/test/java/kafka/testkit/KafkaClusterTestKit.java:
##########
@@ -115,11 +114,33 @@ public void close() {
         }
     }
 
+    static class SimpleFaultHandlerFactory implements FaultHandlerFactory {
+        private final MockFaultHandler fatalFaultHandler = new MockFaultHandler("fatalFaultHandler");
+        private final MockFaultHandler nonFatalFaultHandler = new MockFaultHandler("nonFatalFaultHandler");
+
+        MockFaultHandler fatalFaultHandler() {
+            return fatalFaultHandler;
+        }
+
+        MockFaultHandler nonFatalFaultHandler() {
+            return nonFatalFaultHandler;
+        }
+
+        @Override
+        public FaultHandler build(String name, boolean fatal, Runnable action) {
+            if (fatal) {
+                return fatalFaultHandler;
+            } else {
+                return nonFatalFaultHandler;
+

Review Comment:
   nit: remove newline



##########
core/src/test/java/kafka/testkit/KafkaClusterTestKit.java:
##########
@@ -130,18 +151,56 @@ public Builder setConfigProp(String key, String value) {
             return this;
         }
 
-        public Builder setMetadataFaultHandler(MockFaultHandler metadataFaultHandler) {
-            this.metadataFaultHandler = metadataFaultHandler;
-            return this;
+        private KafkaConfig createNodeConfig(TestKitNode node) {
+            BrokerNode brokerNode = nodes.brokerNodes().get(node.id());
+            ControllerNode controllerNode = nodes.controllerNodes().get(node.id());
+
+            Map<String, String> props = new HashMap<>(configProps);
+            props.put(KafkaConfig$.MODULE$.ProcessRolesProp(), roles(node.id()));
+            props.put(KafkaConfig$.MODULE$.NodeIdProp(),
+                    Integer.toString(node.id()));
+            // In combined mode, always prefer the metadata log directory of the controller node.
+            if (controllerNode != null) {
+                props.put(KafkaConfig$.MODULE$.MetadataLogDirProp(),
+                        controllerNode.metadataDirectory());
+            } else {
+                props.put(KafkaConfig$.MODULE$.MetadataLogDirProp(),
+                        node.metadataDirectory());
+            }
+            // Set the log.dirs according to the broker node setting (if there is a broker node)
+            if (brokerNode != null) {
+                props.put(KafkaConfig$.MODULE$.LogDirsProp(),
+                        String.join(",", brokerNode.logDataDirectories()));
+            }
+            props.put(KafkaConfig$.MODULE$.ListenerSecurityProtocolMapProp(),
+                    "EXTERNAL:PLAINTEXT,CONTROLLER:PLAINTEXT");
+            props.put(KafkaConfig$.MODULE$.ListenersProp(), listeners(node.id()));
+            props.put(KafkaConfig$.MODULE$.InterBrokerListenerNameProp(),
+                    nodes.interBrokerListenerName().value());
+            props.put(KafkaConfig$.MODULE$.ControllerListenerNamesProp(),
+                    "CONTROLLER");
+            // Note: we can't accurately set controller.quorum.voters yet, since we don't
+            // yet know what ports each controller will pick.  Set it to a dummy string \

Review Comment:
   nit: extra `\` at the end of the line intended?



##########
core/src/main/scala/kafka/server/KafkaRaftServer.scala:
##########
@@ -69,95 +64,56 @@ class KafkaRaftServer(
   private val controllerQuorumVotersFuture = CompletableFuture.completedFuture(
     RaftConfig.parseVoterConnections(config.quorumVoters))
 
-  private val raftManager = new KafkaRaftManager[ApiMessageAndVersion](
-    metaProps,
+  private val jointServer = new JointServer(
     config,
-    new MetadataRecordSerde,
-    KafkaRaftServer.MetadataPartition,
-    KafkaRaftServer.MetadataTopicId,
+    metaProps,
     time,
     metrics,
     threadNamePrefix,
-    controllerQuorumVotersFuture
+    controllerQuorumVotersFuture,
+    new StandardFaultHandlerFactory(),
   )
 
   private val broker: Option[BrokerServer] = if (config.processRoles.contains(BrokerRole)) {
-    val brokerMetrics = BrokerServerMetrics(metrics)
-    val fatalFaultHandler = new ProcessExitingFaultHandler()
-    val metadataLoadingFaultHandler = new LoggingFaultHandler("metadata loading",
-        () => brokerMetrics.metadataLoadErrorCount.getAndIncrement())
-    val metadataApplyingFaultHandler = new LoggingFaultHandler("metadata application",
-      () => brokerMetrics.metadataApplyErrorCount.getAndIncrement())
     Some(new BrokerServer(
-      config,
-      metaProps,
-      raftManager,
-      time,
-      metrics,
-      brokerMetrics,
-      threadNamePrefix,
-      offlineDirs,
-      controllerQuorumVotersFuture,
-      fatalFaultHandler,
-      metadataLoadingFaultHandler,
-      metadataApplyingFaultHandler
+      jointServer,
+      offlineDirs
     ))
   } else {
     None
   }
 
   private val controller: Option[ControllerServer] = if (config.processRoles.contains(ControllerRole)) {
-    val controllerMetrics = new QuorumControllerMetrics(KafkaYammerMetrics.defaultRegistry(), time)
-    val metadataFaultHandler = new LoggingFaultHandler("controller metadata",
-      () => controllerMetrics.incrementMetadataErrorCount())
-    val fatalFaultHandler = new ProcessExitingFaultHandler()
     Some(new ControllerServer(
-      metaProps,
-      config,
-      raftManager,
-      time,
-      metrics,
-      controllerMetrics,
-      threadNamePrefix,
-      controllerQuorumVotersFuture,
+      jointServer,
       KafkaRaftServer.configSchema,
-      raftManager.apiVersions,
       bootstrapMetadata,
-      metadataFaultHandler,
-      fatalFaultHandler
     ))
   } else {
     None
   }
 
   override def startup(): Unit = {
     Mx4jLoader.maybeLoad()
-    // Note that we startup `RaftManager` first so that the controller and broker
-    // can register listeners during initialization.
-    raftManager.startup()
     controller.foreach(_.startup())
     broker.foreach(_.startup())
     AppInfoParser.registerAppInfo(Server.MetricsPrefix, config.brokerId.toString, metrics, time.milliseconds())
     info(KafkaBroker.STARTED_MESSAGE)
   }
 
   override def shutdown(): Unit = {
+    // In combined mode, we want to shut down the broker first, since it may take longer to complete.
+    // Additionally, the controller shutdown process stops the raft client early on (currently),
+    // which would disrupt broker shutdown.
     broker.foreach(_.shutdown())
-    // The order of shutdown for `RaftManager` and `ControllerServer` is backwards
-    // compared to `startup()`. This is because the `SocketServer` implementation that
-    // we rely on to receive requests is owned by `ControllerServer`, so we need it
-    // to stick around until graceful shutdown of `RaftManager` can be completed.
-    raftManager.shutdown()
     controller.foreach(_.shutdown())
     CoreUtils.swallow(AppInfoParser.unregisterAppInfo(Server.MetricsPrefix, config.brokerId.toString, metrics), this)

Review Comment:
   Pre-existing issue, but I think we also need to close the `Metrics` instance. 



##########
core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala:
##########
@@ -99,25 +94,28 @@ class KRaftQuorumImplementation(
     startup: Boolean,
     threadNamePrefix: Option[String],
   ): KafkaBroker = {
-    val metrics = new Metrics()
-    val broker = new BrokerServer(config = config,
-      metaProps = new MetaProperties(clusterId, config.nodeId),
-      raftManager = raftManager,
-      time = time,
-      metrics = metrics,
-      brokerMetrics = BrokerServerMetrics(metrics),
-      threadNamePrefix = Some("Broker%02d_".format(config.nodeId)),
-      initialOfflineDirs = Seq(),
-      controllerQuorumVotersFuture = controllerQuorumVotersFuture,
-      fatalFaultHandler = faultHandler,
-      metadataLoadingFaultHandler = faultHandler,
-      metadataPublishingFaultHandler = faultHandler)
-    if (startup) broker.startup()
-    broker
+    val jointServer = new JointServer(config,
+      new MetaProperties(clusterId, config.nodeId),
+      Time.SYSTEM,
+      new Metrics(),
+      Option("Broker%02d_".format(config.nodeId)),
+      controllerQuorumVotersFuture,
+      faultHandlerFactory)
+    var broker: BrokerServer = null
+    try {
+      broker = new BrokerServer(jointServer,
+        initialOfflineDirs = Seq())
+      if (startup) broker.startup()
+      broker
+    } catch {
+      case e: Throwable => {
+        if (broker != null) CoreUtils.swallow(broker.shutdown(), log)

Review Comment:
   Shutdown `jointServer` here if `broker` is null? (Same thing for controller below.)



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

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