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



##########
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:
       20% jitter is kind of a lot for the heartbeat, I think.  20% means we 
could be heartbeating every 2.4 s rather than 2 s.
   
   Thinking about it more I'd be more comfortable with 2%.  The point of jitter 
is just to avoid having a lot of heartbeats come in all at once, and I think 2% 
jitter is more than sufficient to accomplish that.  With a very small injection 
of randomness each time the heartbeat times will drift apart naturally, without 
the need for big divergences.




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to