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 2021/02/10 15:53:53 UTC

[GitHub] [kafka] ijuma commented on a change in pull request #10095: MINOR: Introduce the KIP-500 Broker lifecycle manager

ijuma commented on a change in pull request #10095:
URL: https://github.com/apache/kafka/pull/10095#discussion_r573839484



##########
File path: core/src/main/scala/kafka/server/BrokerLifecycleManager.scala
##########
@@ -0,0 +1,457 @@
+/**
+ * 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, NANOSECONDS}
+import java.util.concurrent.{CompletableFuture, TimeUnit}
+import kafka.utils.Logging
+import org.apache.kafka.clients.ClientResponse
+import org.apache.kafka.common.Uuid
+import org.apache.kafka.common.message.BrokerRegistrationRequestData.ListenerCollection
+import org.apache.kafka.common.message.{BrokerHeartbeatRequestData, BrokerRegistrationRequestData}
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{BrokerHeartbeatRequest, BrokerHeartbeatResponse, BrokerRegistrationRequest, BrokerRegistrationResponse}
+import org.apache.kafka.metadata.{BrokerState, VersionRange}
+import org.apache.kafka.queue.EventQueue.DeadlineFunction
+import org.apache.kafka.common.utils.{ExponentialBackoff, LogContext, Time}
+import org.apache.kafka.queue.{EventQueue, KafkaEventQueue}
+import scala.jdk.CollectionConverters._
+
+
+class BrokerLifecycleManager(val config: KafkaConfig,
+                             val time: Time,
+                             val threadNamePrefix: Option[String]) extends Logging {
+  val logContext = new LogContext(s"[BrokerLifecycleManager id=${config.nodeId}] ")
+
+  this.logIdent = logContext.logPrefix()
+
+  /**
+   * The broker id.
+   */
+  private val nodeId = config.nodeId
+
+  /**
+   * The broker rack, or null if there is no configured rack.
+   */
+  private val rack = config.rack.orNull
+
+  /**
+   * How long to wait for registration to succeed before failing the startup process.
+   */
+  private val initialTimeoutNs = NANOSECONDS.

Review comment:
       Nit: Isn't `TimeUnit.MILLISECONDS.toNanos(config.initialRegistrationTimeoutMs)` more readable?

##########
File path: core/src/main/scala/kafka/server/BrokerLifecycleManager.scala
##########
@@ -0,0 +1,457 @@
+/**
+ * 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, NANOSECONDS}
+import java.util.concurrent.{CompletableFuture, TimeUnit}
+import kafka.utils.Logging
+import org.apache.kafka.clients.ClientResponse
+import org.apache.kafka.common.Uuid
+import org.apache.kafka.common.message.BrokerRegistrationRequestData.ListenerCollection
+import org.apache.kafka.common.message.{BrokerHeartbeatRequestData, BrokerRegistrationRequestData}
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{BrokerHeartbeatRequest, BrokerHeartbeatResponse, BrokerRegistrationRequest, BrokerRegistrationResponse}
+import org.apache.kafka.metadata.{BrokerState, VersionRange}
+import org.apache.kafka.queue.EventQueue.DeadlineFunction
+import org.apache.kafka.common.utils.{ExponentialBackoff, LogContext, Time}
+import org.apache.kafka.queue.{EventQueue, KafkaEventQueue}
+import scala.jdk.CollectionConverters._
+
+
+class BrokerLifecycleManager(val config: KafkaConfig,
+                             val time: Time,
+                             val threadNamePrefix: Option[String]) extends Logging {
+  val logContext = new LogContext(s"[BrokerLifecycleManager id=${config.nodeId}] ")
+
+  this.logIdent = logContext.logPrefix()
+
+  /**
+   * The broker id.
+   */
+  private val nodeId = config.nodeId
+
+  /**
+   * The broker rack, or null if there is no configured rack.
+   */
+  private val rack = config.rack.orNull
+
+  /**
+   * How long to wait for registration to succeed before failing the startup process.
+   */
+  private val initialTimeoutNs = NANOSECONDS.
+    convert(config.initialRegistrationTimeoutMs.longValue(), TimeUnit.MILLISECONDS)
+
+  /**
+   * The exponential backoff to use for resending communication.
+   */
+  private val resendExponentialBackoff =
+    new ExponentialBackoff(100, 2, config.brokerSessionTimeoutMs.toLong, 0.1)

Review comment:
       We seem to use `0.2` as jitter elsewhere. Any reason to deviate? Maybe we should have an `ExponentialBackoff` static factory method that only takes the initial interval and the max interval. It seems that the other two parameters can be the same most of the time.

##########
File path: core/src/main/scala/kafka/server/BrokerLifecycleManager.scala
##########
@@ -0,0 +1,457 @@
+/**
+ * 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, NANOSECONDS}
+import java.util.concurrent.{CompletableFuture, TimeUnit}
+import kafka.utils.Logging
+import org.apache.kafka.clients.ClientResponse
+import org.apache.kafka.common.Uuid
+import org.apache.kafka.common.message.BrokerRegistrationRequestData.ListenerCollection
+import org.apache.kafka.common.message.{BrokerHeartbeatRequestData, BrokerRegistrationRequestData}
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{BrokerHeartbeatRequest, BrokerHeartbeatResponse, BrokerRegistrationRequest, BrokerRegistrationResponse}
+import org.apache.kafka.metadata.{BrokerState, VersionRange}
+import org.apache.kafka.queue.EventQueue.DeadlineFunction
+import org.apache.kafka.common.utils.{ExponentialBackoff, LogContext, Time}
+import org.apache.kafka.queue.{EventQueue, KafkaEventQueue}
+import scala.jdk.CollectionConverters._
+
+
+class BrokerLifecycleManager(val config: KafkaConfig,
+                             val time: Time,
+                             val threadNamePrefix: Option[String]) extends Logging {
+  val logContext = new LogContext(s"[BrokerLifecycleManager id=${config.nodeId}] ")
+
+  this.logIdent = logContext.logPrefix()
+
+  /**
+   * The broker id.
+   */
+  private val nodeId = config.nodeId
+
+  /**
+   * The broker rack, or null if there is no configured rack.
+   */
+  private val rack = config.rack.orNull
+
+  /**
+   * How long to wait for registration to succeed before failing the startup process.
+   */
+  private val initialTimeoutNs = NANOSECONDS.
+    convert(config.initialRegistrationTimeoutMs.longValue(), TimeUnit.MILLISECONDS)
+
+  /**
+   * The exponential backoff to use for resending communication.
+   */
+  private val resendExponentialBackoff =
+    new ExponentialBackoff(100, 2, config.brokerSessionTimeoutMs.toLong, 0.1)
+
+  /**
+   * The number of tries we've tried to communicate.
+   */
+  private var failedAttempts = 0L
+
+  /**
+   * The broker incarnation ID.  This ID uniquely identifies each time we start the broker
+   */
+  val incarnationId = Uuid.randomUuid()
+
+  /**
+   * A future which is completed just as soon as the broker has caught up with the latest
+   * metadata offset for the first time.
+   */
+  val initialCatchUpFuture = new CompletableFuture[Void]()
+
+  /**
+   * A future which is completed when controlled shutdown is done.
+   */
+  val controlledShutdownFuture = new CompletableFuture[Void]()
+
+  /**
+   * The broker epoch, or -1 if the broker has not yet registered.
+   * This variable can only be written from the event queue thread.
+   */
+  @volatile private var _brokerEpoch = -1L
+
+  /**
+   * The current broker state.
+   * This variable can only be written from the event queue thread.
+   */
+  @volatile private var _state = BrokerState.NOT_RUNNING
+
+  /**
+   * A callback function which gives this manager the current highest metadata offset.
+   * This function must be thread-safe.
+   */
+  private var _highestMetadataOffsetProvider: () => Long = null
+
+  /**
+   * True only if we are ready to unfence the broker.
+   * This variable can only be accessed from the event queue thread.
+   */
+  private var readyToUnfence = false
+
+  /**
+   * True if we sent a heartbeat to the active controller requesting controlled
+   * shutdown.
+   */
+  private var gotControlledShutdownResponse = false
+
+  /**
+   * Whether or not we this broker is registered with the controller quorum.
+   * This variable can only be accessed from the event queue thread.
+   */
+  private var registered = false
+
+  /**
+   * True if the initial registration succeeded.
+   * This variable can only be accessed from the event queue thread.
+   */
+  private var initialRegistrationSucceeded = false
+
+  /**
+   * The cluster ID, or null if this manager has not been started yet.
+   * This variable can only be accessed from the event queue thread.
+   */
+  private var _clusterId: Uuid = null
+
+  /**
+   * The listeners which this broker advertises.
+   */
+  private var _advertisedListeners: ListenerCollection = null
+
+  /**
+   * The features supported by this broker.
+   */
+  private var _supportedFeatures: util.Map[String, VersionRange] = null
+
+  /**
+   * The channel manager, or null if this manager has not been started yet.
+   * This variable can only be accessed from the event queue thread.
+   */
+  var _channelManager: BrokerToControllerChannelManager = null
+
+  /**
+   * The event queue.
+   */
+  val eventQueue = new KafkaEventQueue(time, logContext, threadNamePrefix.getOrElse(""))
+
+  /**
+   * Start the BrokerLifecycleManager.
+   *
+   * @param highestMetadataOffsetProvider Provides the current highest metadata offset.
+   * @param channelManager                The brokerToControllerChannelManager to use.
+   * @param clusterId                     The cluster ID.
+   */
+  def start(highestMetadataOffsetProvider: () => Long,
+            channelManager: BrokerToControllerChannelManager,
+            clusterId: Uuid,
+            advertisedListeners: ListenerCollection,
+            supportedFeatures: util.Map[String, VersionRange]): Unit = {

Review comment:
       Why are we using a Java collection?

##########
File path: core/src/main/scala/kafka/server/BrokerLifecycleManager.scala
##########
@@ -0,0 +1,457 @@
+/**
+ * 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, NANOSECONDS}
+import java.util.concurrent.{CompletableFuture, TimeUnit}
+import kafka.utils.Logging
+import org.apache.kafka.clients.ClientResponse
+import org.apache.kafka.common.Uuid
+import org.apache.kafka.common.message.BrokerRegistrationRequestData.ListenerCollection
+import org.apache.kafka.common.message.{BrokerHeartbeatRequestData, BrokerRegistrationRequestData}
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{BrokerHeartbeatRequest, BrokerHeartbeatResponse, BrokerRegistrationRequest, BrokerRegistrationResponse}
+import org.apache.kafka.metadata.{BrokerState, VersionRange}
+import org.apache.kafka.queue.EventQueue.DeadlineFunction
+import org.apache.kafka.common.utils.{ExponentialBackoff, LogContext, Time}
+import org.apache.kafka.queue.{EventQueue, KafkaEventQueue}
+import scala.jdk.CollectionConverters._
+
+
+class BrokerLifecycleManager(val config: KafkaConfig,

Review comment:
       Worth adding a comment explaining the thread safety aspects of this class and also its general purpose. I see a few volatile fields, but `failedAttempts` is not volatile (for example).

##########
File path: core/src/main/scala/kafka/server/BrokerLifecycleManager.scala
##########
@@ -0,0 +1,457 @@
+/**
+ * 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, NANOSECONDS}
+import java.util.concurrent.{CompletableFuture, TimeUnit}
+import kafka.utils.Logging
+import org.apache.kafka.clients.ClientResponse
+import org.apache.kafka.common.Uuid
+import org.apache.kafka.common.message.BrokerRegistrationRequestData.ListenerCollection
+import org.apache.kafka.common.message.{BrokerHeartbeatRequestData, BrokerRegistrationRequestData}
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{BrokerHeartbeatRequest, BrokerHeartbeatResponse, BrokerRegistrationRequest, BrokerRegistrationResponse}
+import org.apache.kafka.metadata.{BrokerState, VersionRange}
+import org.apache.kafka.queue.EventQueue.DeadlineFunction
+import org.apache.kafka.common.utils.{ExponentialBackoff, LogContext, Time}
+import org.apache.kafka.queue.{EventQueue, KafkaEventQueue}
+import scala.jdk.CollectionConverters._
+
+
+class BrokerLifecycleManager(val config: KafkaConfig,
+                             val time: Time,
+                             val threadNamePrefix: Option[String]) extends Logging {
+  val logContext = new LogContext(s"[BrokerLifecycleManager id=${config.nodeId}] ")
+
+  this.logIdent = logContext.logPrefix()
+
+  /**
+   * The broker id.
+   */
+  private val nodeId = config.nodeId
+
+  /**
+   * The broker rack, or null if there is no configured rack.
+   */
+  private val rack = config.rack.orNull
+
+  /**
+   * How long to wait for registration to succeed before failing the startup process.
+   */
+  private val initialTimeoutNs = NANOSECONDS.
+    convert(config.initialRegistrationTimeoutMs.longValue(), TimeUnit.MILLISECONDS)
+
+  /**
+   * The exponential backoff to use for resending communication.
+   */
+  private val resendExponentialBackoff =
+    new ExponentialBackoff(100, 2, config.brokerSessionTimeoutMs.toLong, 0.1)
+
+  /**
+   * The number of tries we've tried to communicate.
+   */
+  private var failedAttempts = 0L
+
+  /**
+   * The broker incarnation ID.  This ID uniquely identifies each time we start the broker
+   */
+  val incarnationId = Uuid.randomUuid()
+
+  /**
+   * A future which is completed just as soon as the broker has caught up with the latest
+   * metadata offset for the first time.
+   */
+  val initialCatchUpFuture = new CompletableFuture[Void]()
+
+  /**
+   * A future which is completed when controlled shutdown is done.
+   */
+  val controlledShutdownFuture = new CompletableFuture[Void]()
+
+  /**
+   * The broker epoch, or -1 if the broker has not yet registered.
+   * This variable can only be written from the event queue thread.
+   */
+  @volatile private var _brokerEpoch = -1L
+
+  /**
+   * The current broker state.
+   * This variable can only be written from the event queue thread.
+   */
+  @volatile private var _state = BrokerState.NOT_RUNNING
+
+  /**
+   * A callback function which gives this manager the current highest metadata offset.
+   * This function must be thread-safe.
+   */
+  private var _highestMetadataOffsetProvider: () => Long = null
+
+  /**
+   * True only if we are ready to unfence the broker.
+   * This variable can only be accessed from the event queue thread.
+   */
+  private var readyToUnfence = false
+
+  /**
+   * True if we sent a heartbeat to the active controller requesting controlled
+   * shutdown.
+   */
+  private var gotControlledShutdownResponse = false
+
+  /**
+   * Whether or not we this broker is registered with the controller quorum.
+   * This variable can only be accessed from the event queue thread.
+   */
+  private var registered = false
+
+  /**
+   * True if the initial registration succeeded.
+   * This variable can only be accessed from the event queue thread.
+   */
+  private var initialRegistrationSucceeded = false
+
+  /**
+   * The cluster ID, or null if this manager has not been started yet.
+   * This variable can only be accessed from the event queue thread.
+   */
+  private var _clusterId: Uuid = null
+
+  /**
+   * The listeners which this broker advertises.
+   */
+  private var _advertisedListeners: ListenerCollection = null
+
+  /**
+   * The features supported by this broker.
+   */
+  private var _supportedFeatures: util.Map[String, VersionRange] = null
+
+  /**
+   * The channel manager, or null if this manager has not been started yet.
+   * This variable can only be accessed from the event queue thread.
+   */
+  var _channelManager: BrokerToControllerChannelManager = null
+
+  /**
+   * The event queue.
+   */
+  val eventQueue = new KafkaEventQueue(time, logContext, threadNamePrefix.getOrElse(""))
+
+  /**
+   * Start the BrokerLifecycleManager.
+   *
+   * @param highestMetadataOffsetProvider Provides the current highest metadata offset.
+   * @param channelManager                The brokerToControllerChannelManager to use.
+   * @param clusterId                     The cluster ID.
+   */
+  def start(highestMetadataOffsetProvider: () => Long,
+            channelManager: BrokerToControllerChannelManager,
+            clusterId: Uuid,
+            advertisedListeners: ListenerCollection,
+            supportedFeatures: util.Map[String, VersionRange]): Unit = {
+    eventQueue.append(new StartupEvent(highestMetadataOffsetProvider,
+      channelManager, clusterId, advertisedListeners, supportedFeatures))
+  }
+
+  def setReadyToUnfence(): Unit = {
+    eventQueue.append(new SetReadyToUnfenceEvent())
+  }
+
+  def brokerEpoch(): Long = _brokerEpoch
+
+  def state(): BrokerState = _state
+
+  class BeginControlledShutdownEvent extends EventQueue.Event {

Review comment:
       Are these events meant to be public?




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

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