in core/src/main/scala/kafka/server/ControllerServer.scala [123:432]
def startup(): Unit = {
if (!maybeChangeStatus(SHUTDOWN, STARTING)) return
val startupDeadline = Deadline.fromDelay(time, config.serverMaxStartupTimeMs, TimeUnit.MILLISECONDS)
try {
this.logIdent = logContext.logPrefix()
info("Starting controller")
config.dynamicConfig.initialize(clientMetricsReceiverPluginOpt = None)
maybeChangeStatus(STARTING, STARTED)
metricsGroup.newGauge("ClusterId", () => clusterId)
metricsGroup.newGauge("yammer-metrics-count", () => KafkaYammerMetrics.defaultRegistry.allMetrics.size)
linuxIoMetricsCollector = new LinuxIoMetricsCollector("/proc", time)
if (linuxIoMetricsCollector.usable()) {
metricsGroup.newGauge("linux-disk-read-bytes", () => linuxIoMetricsCollector.readBytes())
metricsGroup.newGauge("linux-disk-write-bytes", () => linuxIoMetricsCollector.writeBytes())
}
authorizerPlugin = config.createNewAuthorizer(metrics, ProcessRole.ControllerRole.toString)
metadataCache = new KRaftMetadataCache(config.nodeId, () => raftManager.client.kraftVersion())
metadataCachePublisher = new KRaftMetadataCachePublisher(metadataCache)
featuresPublisher = new FeaturesPublisher(logContext)
registrationsPublisher = new ControllerRegistrationsPublisher()
incarnationId = Uuid.randomUuid()
val apiVersionManager = new SimpleApiVersionManager(
ListenerType.CONTROLLER,
config.unstableApiVersionsEnabled,
() => featuresPublisher.features().setFinalizedLevel(
KRaftVersion.FEATURE_NAME,
raftManager.client.kraftVersion().featureLevel())
)
// metrics will be set to null when closing a controller, so we should recreate it for testing
if (sharedServer.metrics == null){
sharedServer.metrics = new Metrics()
}
tokenCache = new DelegationTokenCache(ScramMechanism.mechanismNames)
credentialProvider = new CredentialProvider(ScramMechanism.mechanismNames, tokenCache)
socketServer = new SocketServer(config,
metrics,
time,
credentialProvider,
apiVersionManager,
sharedServer.socketFactory)
val listenerInfo = ListenerInfo
.create(config.effectiveAdvertisedControllerListeners.asJava)
.withWildcardHostnamesResolved()
.withEphemeralPortsCorrected(name => socketServer.boundPort(new ListenerName(name)))
socketServerFirstBoundPortFuture.complete(listenerInfo.firstListener().port())
val endpointReadyFutures = {
val builder = new EndpointReadyFutures.Builder()
builder.build(authorizerPlugin.toJava,
new KafkaAuthorizerServerInfo(
new ClusterResource(clusterId),
config.nodeId,
listenerInfo.listeners().values(),
listenerInfo.firstListener(),
config.earlyStartListeners.map(_.value()).asJava))
}
sharedServer.startForController(listenerInfo)
createTopicPolicy = Option(config.
getConfiguredInstance(CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, classOf[CreateTopicPolicy]))
alterConfigPolicy = Option(config.
getConfiguredInstance(ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, classOf[AlterConfigPolicy]))
val voterConnections = FutureUtils.waitWithLogging(logger.underlying, logIdent,
"controller quorum voters future",
sharedServer.controllerQuorumVotersFuture,
startupDeadline, time)
val controllerNodes = QuorumConfig.voterConnectionsToNodes(voterConnections)
val quorumFeatures = new QuorumFeatures(config.nodeId,
QuorumFeatures.defaultSupportedFeatureMap(config.unstableFeatureVersionsEnabled),
controllerNodes.asScala.map(node => Integer.valueOf(node.id())).asJava)
val delegationTokenManagerConfigs = new DelegationTokenManagerConfigs(config)
val delegationTokenKeyString = {
if (delegationTokenManagerConfigs.tokenAuthEnabled) {
delegationTokenManagerConfigs.delegationTokenSecretKey.value
} else {
null
}
}
val controllerBuilder = {
val leaderImbalanceCheckIntervalNs = if (config.autoLeaderRebalanceEnable) {
OptionalLong.of(TimeUnit.NANOSECONDS.convert(config.leaderImbalanceCheckIntervalSeconds, TimeUnit.SECONDS))
} else {
OptionalLong.empty()
}
val maxIdleIntervalNs = config.metadataMaxIdleIntervalNs.fold(OptionalLong.empty)(OptionalLong.of)
quorumControllerMetrics = new QuorumControllerMetrics(Optional.of(KafkaYammerMetrics.defaultRegistry), time)
new QuorumController.Builder(config.nodeId, sharedServer.clusterId).
setTime(time).
setThreadNamePrefix(s"quorum-controller-${config.nodeId}-").
setConfigSchema(configSchema).
setRaftClient(raftManager.client).
setQuorumFeatures(quorumFeatures).
setDefaultReplicationFactor(config.defaultReplicationFactor.toShort).
setDefaultNumPartitions(config.numPartitions.intValue()).
setSessionTimeoutNs(TimeUnit.NANOSECONDS.convert(config.brokerSessionTimeoutMs.longValue(),
TimeUnit.MILLISECONDS)).
setLeaderImbalanceCheckIntervalNs(leaderImbalanceCheckIntervalNs).
setMaxIdleIntervalNs(maxIdleIntervalNs).
setMetrics(quorumControllerMetrics).
setCreateTopicPolicy(createTopicPolicy.toJava).
setAlterConfigPolicy(alterConfigPolicy.toJava).
setConfigurationValidator(new ControllerConfigurationValidator(sharedServer.brokerConfig)).
setStaticConfig(config.originals).
setBootstrapMetadata(bootstrapMetadata).
setFatalFaultHandler(sharedServer.fatalQuorumControllerFaultHandler).
setNonFatalFaultHandler(sharedServer.nonFatalQuorumControllerFaultHandler).
setDelegationTokenCache(tokenCache).
setDelegationTokenSecretKey(delegationTokenKeyString).
setDelegationTokenMaxLifeMs(delegationTokenManagerConfigs.delegationTokenMaxLifeMs).
setDelegationTokenExpiryTimeMs(delegationTokenManagerConfigs.delegationTokenExpiryTimeMs).
setDelegationTokenExpiryCheckIntervalMs(delegationTokenManagerConfigs.delegationTokenExpiryCheckIntervalMs).
setUncleanLeaderElectionCheckIntervalMs(config.uncleanLeaderElectionCheckIntervalMs).
setInterBrokerListenerName(config.interBrokerListenerName.value()).
setControllerPerformanceSamplePeriodMs(config.controllerPerformanceSamplePeriodMs).
setControllerPerformanceAlwaysLogThresholdMs(config.controllerPerformanceAlwaysLogThresholdMs)
}
controller = controllerBuilder.build()
// If we are using a ClusterMetadataAuthorizer, requests to add or remove ACLs must go
// through the controller.
authorizerPlugin.foreach { plugin =>
plugin.get match {
case a: ClusterMetadataAuthorizer => a.setAclMutator(controller)
case _ =>
}
}
quotaManagers = QuotaFactory.instantiate(config,
metrics,
time,
s"controller-${config.nodeId}-", ProcessRole.ControllerRole.toString)
clientQuotaMetadataManager = new ClientQuotaMetadataManager(quotaManagers, socketServer.connectionQuotas)
controllerApis = new ControllerApis(socketServer.dataPlaneRequestChannel,
authorizerPlugin,
quotaManagers,
time,
controller,
raftManager,
config,
clusterId,
registrationsPublisher,
apiVersionManager,
metadataCache)
controllerApisHandlerPool = new KafkaRequestHandlerPool(config.nodeId,
socketServer.dataPlaneRequestChannel,
controllerApis,
time,
config.numIoThreads,
"RequestHandlerAvgIdlePercent",
"controller")
// Set up the metadata cache publisher.
metadataPublishers.add(metadataCachePublisher)
// Set up the metadata features publisher.
metadataPublishers.add(featuresPublisher)
// Set up the controller registrations publisher.
metadataPublishers.add(registrationsPublisher)
// Create the registration manager, which handles sending KIP-919 controller registrations.
registrationManager = new ControllerRegistrationManager(config.nodeId,
clusterId,
time,
s"controller-${config.nodeId}-",
QuorumFeatures.defaultSupportedFeatureMap(config.unstableFeatureVersionsEnabled),
incarnationId,
listenerInfo)
// Add the registration manager to the list of metadata publishers, so that it receives
// callbacks when the cluster registrations change.
metadataPublishers.add(registrationManager)
// Set up the dynamic config publisher. This runs even in combined mode, since the broker
// has its own separate dynamic configuration object.
metadataPublishers.add(new DynamicConfigPublisher(
config,
sharedServer.metadataPublishingFaultHandler,
immutable.Map[ConfigType, ConfigHandler](
// controllers don't host topics, so no need to do anything with dynamic topic config changes here
ConfigType.BROKER -> new BrokerConfigHandler(config, quotaManagers)
),
"controller"))
// Register this instance for dynamic config changes to the KafkaConfig. This must be called
// after the authorizer and quotaManagers are initialized, since it references those objects.
// It must be called before DynamicClientQuotaPublisher is installed, since otherwise we may
// miss the initial update which establishes the dynamic configurations that are in effect on
// startup.
config.dynamicConfig.addReconfigurables(this)
// Set up the client quotas publisher. This will enable controller mutation quotas and any
// other quotas which are applicable.
metadataPublishers.add(new DynamicClientQuotaPublisher(
config,
sharedServer.metadataPublishingFaultHandler,
"controller",
clientQuotaMetadataManager
))
// Set up the DynamicTopicClusterQuotaPublisher. This will enable quotas for the cluster and topics.
metadataPublishers.add(new DynamicTopicClusterQuotaPublisher(
clusterId,
config,
sharedServer.metadataPublishingFaultHandler,
"controller",
quotaManagers,
))
// Set up the SCRAM publisher.
metadataPublishers.add(new ScramPublisher(
config,
sharedServer.metadataPublishingFaultHandler,
"controller",
credentialProvider
))
// Set up the DelegationToken publisher.
// We need a tokenManager for the Publisher
// The tokenCache in the tokenManager is the same used in DelegationTokenControlManager
metadataPublishers.add(new DelegationTokenPublisher(
config,
sharedServer.metadataPublishingFaultHandler,
"controller",
new DelegationTokenManager(delegationTokenManagerConfigs, tokenCache)
))
// Set up the metrics publisher.
metadataPublishers.add(new ControllerMetadataMetricsPublisher(
sharedServer.controllerServerMetrics,
sharedServer.metadataPublishingFaultHandler
))
// Set up the ACL publisher.
metadataPublishers.add(new AclPublisher(
config.nodeId,
sharedServer.metadataPublishingFaultHandler,
"controller",
authorizerPlugin.toJava
))
// Install all metadata publishers.
FutureUtils.waitWithLogging(logger.underlying, logIdent,
"the controller metadata publishers to be installed",
sharedServer.loader.installPublishers(metadataPublishers), startupDeadline, time)
val authorizerFutures: Map[Endpoint, CompletableFuture[Void]] = endpointReadyFutures.futures().asScala.toMap
/**
* Enable the controller endpoint(s). If we are using an authorizer which stores
* ACLs in the metadata log, such as StandardAuthorizer, we will be able to start
* accepting requests from principals included super.users right after this point,
* but we will not be able to process requests from non-superusers until AclPublisher
* publishes metadata from the QuorumController. MetadataPublishers do not publish
* metadata until the controller has caught up to the high watermark.
*/
val socketServerFuture = socketServer.enableRequestProcessing(authorizerFutures)
/**
* Start the KIP-919 controller registration manager.
*/
val controllerNodeProvider = RaftControllerNodeProvider(raftManager, config)
registrationChannelManager = new NodeToControllerChannelManagerImpl(
controllerNodeProvider,
time,
metrics,
config,
"registration",
s"controller-${config.nodeId}-",
5000)
registrationChannelManager.start()
registrationManager.start(registrationChannelManager)
// Block here until all the authorizer futures are complete
FutureUtils.waitWithLogging(logger.underlying, logIdent,
"all of the authorizer futures to be completed",
CompletableFuture.allOf(authorizerFutures.values.toSeq: _*), startupDeadline, time)
// Wait for all the SocketServer ports to be open, and the Acceptors to be started.
FutureUtils.waitWithLogging(logger.underlying, logIdent,
"all of the SocketServer Acceptors to be started",
socketServerFuture, startupDeadline, time)
} catch {
case e: Throwable =>
maybeChangeStatus(STARTING, STARTED)
sharedServer.controllerStartupFaultHandler.handleFault("caught exception", e)
shutdown()
throw e
}
}