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 16:43:31 UTC

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

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



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

Review comment:
       nit: append " and failed" or `s/tried/failed/`

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

Review comment:
       nit: `s/we this/this/`

##########
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:
       I wonder if it might be helpful to assign each `var` into one of two buckets: stuff that can only be written to from the event queue thread but that can be read from any thread (must be `@volatile`), and stuff that is only used from within the event queue (doesn't need to be).  At a minimum put these into different commented sections, but maybe even create a single container object for the `@volatile` ones:
   
   
   ```
     private case class EventQueueThreadOwnedVars(@volatile var _brokerEpoch: Long = -1L,
                                                  @volatile var _state: BrokerState = BrokerState.NOT_RUNNING)
     val eventQueueThreadOwnedVars = EventQueueThreadOwnedVars()
   ```
   

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

Review comment:
       Make an `Option[String]`?  No need to propagate the unfortunate fact that rack can be null elsewhere?

##########
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 {
+    override def run(): Unit = {
+      _state match {
+        case BrokerState.PENDING_CONTROLLED_SHUTDOWN =>
+          info(s"Attempted to enter controlled shutdown state, but we are already in " +

Review comment:
       nit: `s/controlled/pending controlled/`

##########
File path: core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala
##########
@@ -0,0 +1,217 @@
+/**
+ * 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.{Collections, Properties}
+import java.util.concurrent.atomic.{AtomicLong, AtomicReference}
+
+import kafka.utils.{MockTime, TestUtils}
+import org.apache.kafka.clients.{Metadata, MockClient, NodeApiVersions}
+import org.apache.kafka.common.{Node, Uuid}
+import org.apache.kafka.common.internals.ClusterResourceListeners
+import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion
+import org.apache.kafka.common.message.BrokerRegistrationRequestData.{Listener, ListenerCollection}
+import org.apache.kafka.common.message.{BrokerHeartbeatResponseData, BrokerRegistrationResponseData}
+import org.apache.kafka.common.network.ListenerName
+import org.apache.kafka.common.protocol.ApiKeys.{BROKER_HEARTBEAT, BROKER_REGISTRATION}
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{BrokerHeartbeatResponse, BrokerRegistrationResponse}
+import org.apache.kafka.common.security.auth.SecurityProtocol
+import org.apache.kafka.common.utils.LogContext
+import org.apache.kafka.metadata.BrokerState
+import org.junit.jupiter.api.{Assertions, Test, Timeout}
+
+import scala.jdk.CollectionConverters._
+
+
+@Timeout(value = 12)
+class BrokerLifecycleManagerTest {
+  def configProperties = {
+    val properties = new Properties()
+    properties.setProperty(KafkaConfig.LogDirsProp, "/tmp/foo")
+    properties.setProperty(KafkaConfig.ProcessRolesProp, "broker")
+    properties.setProperty(KafkaConfig.NodeIdProp, "1")
+    properties.setProperty(KafkaConfig.InitialBrokerRegistrationTimeoutMs, "300000")
+    properties
+  }
+
+  class SimpleControllerNodeProvider extends ControllerNodeProvider {
+    val node = new AtomicReference[Node](null)
+
+    override def get(): Option[Node] = Option(node.get())
+
+    override def listenerName: ListenerName = new ListenerName("PLAINTEXT")
+
+    override def securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT;
+  }
+
+  class BrokerLifecycleManagerTestContext(properties: Properties) {
+    val config = new KafkaConfig(properties)
+    val time = new MockTime(1, 1)
+    val highestMetadataOffset = new AtomicLong(0)
+    val metadata = new Metadata(1000, 1000, new LogContext(), new ClusterResourceListeners())
+    val mockClient = new MockClient(time, metadata)
+    val controllerNodeProvider = new SimpleControllerNodeProvider()
+    val nodeApiVersions = new NodeApiVersions(Seq(BROKER_REGISTRATION, BROKER_HEARTBEAT).map {
+      apiKey => new ApiVersion().setApiKey(apiKey.id).
+        setMinVersion(apiKey.oldestVersion()).setMaxVersion(apiKey.latestVersion())
+    }.toList.asJava)
+    val mockChannelManager = new MockBrokerToControllerChannelManager(mockClient,
+      time, controllerNodeProvider, nodeApiVersions)
+    val clusterId = Uuid.fromString("x4AJGXQSRnephtTZzujw4w")
+    val advertisedListeners = new ListenerCollection()
+    config.advertisedListeners.foreach { ep =>
+      advertisedListeners.add(new Listener().setHost(ep.host).
+        setName(ep.listenerName.value()).
+        setPort(ep.port.shortValue()).
+        setSecurityProtocol(ep.securityProtocol.id))
+    }
+
+    def poll(): Unit = {
+      mockClient.wakeup()
+      mockChannelManager.poll()
+    }
+  }
+
+  @Test
+  def testCreateAndClose(): Unit = {
+    val context = new BrokerLifecycleManagerTestContext(configProperties)
+    val manager = new BrokerLifecycleManager(context.config, context.time, None)
+    manager.close()
+  }
+
+  @Test
+  def testCreateStartAndClose(): Unit = {
+    val context = new BrokerLifecycleManagerTestContext(configProperties)
+    val manager = new BrokerLifecycleManager(context.config, context.time, None)
+    Assertions.assertEquals(BrokerState.NOT_RUNNING, manager.state())
+    manager.start(() => context.highestMetadataOffset.get(),
+      context.mockChannelManager, context.clusterId, context.advertisedListeners,
+      Collections.emptyMap())
+    TestUtils.retry(60000) {
+      Assertions.assertEquals(BrokerState.STARTING, manager.state())
+    }
+    manager.close()
+    Assertions.assertEquals(BrokerState.SHUTTING_DOWN, manager.state())
+  }
+
+  @Test
+  def testSuccessfulRegistration(): Unit = {
+    val context = new BrokerLifecycleManagerTestContext(configProperties)
+    val manager = new BrokerLifecycleManager(context.config, context.time, None)
+    val controllerNode = new Node(3000, "localhost", 8021)
+    context.controllerNodeProvider.node.set(controllerNode)
+    context.mockClient.prepareResponseFrom(new BrokerRegistrationResponse(
+      new BrokerRegistrationResponseData().setBrokerEpoch(1000)), controllerNode)
+    manager.start(() => context.highestMetadataOffset.get(),
+      context.mockChannelManager, context.clusterId, context.advertisedListeners,
+      Collections.emptyMap())
+    TestUtils.retry(10000) {
+      context.poll()
+      Assertions.assertEquals(1000L, manager.brokerEpoch())
+    }
+    manager.close()
+
+  }
+
+  @Test
+  def testRegistrationTimeout(): Unit = {
+    val context = new BrokerLifecycleManagerTestContext(configProperties)
+    val controllerNode = new Node(3000, "localhost", 8021)
+    val manager = new BrokerLifecycleManager(context.config, context.time, None)
+    context.controllerNodeProvider.node.set(controllerNode)
+    def newDuplicateRegistrationResponse(): Unit = {
+      context.mockClient.prepareResponseFrom(new BrokerRegistrationResponse(
+        new BrokerRegistrationResponseData().
+          setErrorCode(Errors.DUPLICATE_BROKER_REGISTRATION.code())), controllerNode)
+      context.mockChannelManager.poll()
+    }
+    newDuplicateRegistrationResponse()
+    Assertions.assertEquals(1, context.mockClient.futureResponses().size)
+    manager.start(() => context.highestMetadataOffset.get(),
+      context.mockChannelManager, context.clusterId, context.advertisedListeners,
+      Collections.emptyMap())
+    // We should send the first registration request and get a failure immediately
+    TestUtils.retry(60000) {
+      context.poll()
+      Assertions.assertEquals(0, context.mockClient.futureResponses().size)
+    }
+    // Verify that we resend the registration request.
+    newDuplicateRegistrationResponse()
+    TestUtils.retry(60000) {
+      context.time.sleep(100)
+      context.poll()
+      manager.eventQueue.wakeup()
+      Assertions.assertEquals(0, context.mockClient.futureResponses().size)
+    }
+    // Verify that we time out eventually.
+    context.time.sleep(300000)
+    TestUtils.retry(60000) {
+      context.poll()
+      manager.eventQueue.wakeup()
+      Assertions.assertEquals(BrokerState.SHUTTING_DOWN, manager.state())
+      Assertions.assertTrue(manager.initialCatchUpFuture.isCompletedExceptionally())
+      Assertions.assertEquals(-1L, manager.brokerEpoch())
+    }
+    manager.close()
+  }
+
+  @Test
+  def testControlledShutdown(): Unit = {
+    val context = new BrokerLifecycleManagerTestContext(configProperties)
+    val manager = new BrokerLifecycleManager(context.config, context.time, None)
+    val controllerNode = new Node(3000, "localhost", 8021)
+    context.controllerNodeProvider.node.set(controllerNode)
+    context.mockClient.prepareResponseFrom(new BrokerRegistrationResponse(
+      new BrokerRegistrationResponseData().setBrokerEpoch(1000)), controllerNode)
+    context.mockClient.prepareResponseFrom(new BrokerHeartbeatResponse(
+      new BrokerHeartbeatResponseData().setIsCaughtUp(true)), controllerNode)
+    manager.start(() => context.highestMetadataOffset.get(),
+      context.mockChannelManager, context.clusterId, context.advertisedListeners,
+      Collections.emptyMap())
+    TestUtils.retry(10000) {
+      context.poll()
+      manager.eventQueue.wakeup()
+      Assertions.assertEquals(BrokerState.RECOVERY, manager.state())
+    }
+    context.mockClient.prepareResponseFrom(new BrokerHeartbeatResponse(
+      new BrokerHeartbeatResponseData().setIsFenced(false)), controllerNode)
+    context.time.sleep(20)
+    TestUtils.retry(10000) {
+      context.poll()
+      manager.eventQueue.wakeup()
+      Assertions.assertEquals(BrokerState.RUNNING, manager.state())
+    }
+    manager.beginControlledShutdown()
+    TestUtils.retry(10000) {
+      context.poll()
+      manager.eventQueue.wakeup()
+      Assertions.assertEquals(BrokerState.PENDING_CONTROLLED_SHUTDOWN, manager.state())
+    }
+    context.mockClient.prepareResponseFrom(new BrokerHeartbeatResponse(
+      new BrokerHeartbeatResponseData().setShouldShutDown(true)), controllerNode)
+    context.time.sleep(3000)
+    TestUtils.retry(10000) {
+      context.poll()
+      manager.eventQueue.wakeup()
+      Assertions.assertEquals(BrokerState.SHUTTING_DOWN, manager.state())
+    }
+    manager.controlledShutdownFuture.get()
+    manager.close()
+  }
+}

Review comment:
       nit: missing newline




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