-
Notifications
You must be signed in to change notification settings - Fork 13.6k
/
BrokerServer.scala
694 lines (586 loc) · 28.7 KB
/
BrokerServer.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
/**
* 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.cluster.EndPoint
import kafka.coordinator.group.{CoordinatorLoaderImpl, CoordinatorPartitionWriter, GroupCoordinatorAdapter}
import kafka.coordinator.transaction.{ProducerIdManager, TransactionCoordinator}
import kafka.log.LogManager
import kafka.log.remote.RemoteLogManager
import kafka.network.{DataPlaneAcceptor, SocketServer}
import kafka.raft.KafkaRaftManager
import kafka.security.CredentialProvider
import kafka.server.metadata.{BrokerMetadataPublisher, ClientQuotaMetadataManager, DynamicClientQuotaPublisher, DynamicConfigPublisher, KRaftMetadataCache, ScramPublisher}
import kafka.utils.CoreUtils
import org.apache.kafka.clients.NetworkClient
import org.apache.kafka.common.config.ConfigException
import org.apache.kafka.common.feature.SupportedVersionRange
import org.apache.kafka.common.message.ApiMessageType.ListenerType
import org.apache.kafka.common.message.BrokerRegistrationRequestData.{Listener, ListenerCollection}
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.security.scram.internals.ScramMechanism
import org.apache.kafka.common.security.token.delegation.internals.DelegationTokenCache
import org.apache.kafka.common.utils.{LogContext, Time, Utils}
import org.apache.kafka.common.{ClusterResource, Endpoint, KafkaException, TopicPartition}
import org.apache.kafka.coordinator.group
import org.apache.kafka.coordinator.group.util.SystemTimerReaper
import org.apache.kafka.coordinator.group.{GroupCoordinator, GroupCoordinatorConfig, GroupCoordinatorService, RecordSerde}
import org.apache.kafka.image.publisher.MetadataPublisher
import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer
import org.apache.kafka.metadata.{BrokerState, VersionRange}
import org.apache.kafka.raft.RaftConfig
import org.apache.kafka.server.authorizer.Authorizer
import org.apache.kafka.server.common.ApiMessageAndVersion
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.apache.kafka.server.network.{EndpointReadyFutures, KafkaAuthorizerServerInfo}
import org.apache.kafka.server.util.timer.SystemTimer
import org.apache.kafka.server.util.{Deadline, FutureUtils, KafkaScheduler}
import org.apache.kafka.storage.internals.log.LogDirFailureChannel
import java.net.InetAddress
import java.util
import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.locks.ReentrantLock
import java.util.concurrent.{CompletableFuture, ExecutionException, TimeUnit, TimeoutException}
import scala.collection.{Map, Seq}
import scala.compat.java8.OptionConverters.RichOptionForJava8
import scala.jdk.CollectionConverters._
/**
* A Kafka broker that runs in KRaft (Kafka Raft) mode.
*/
class BrokerServer(
val sharedServer: SharedServer,
val initialOfflineDirs: Seq[String],
) extends KafkaBroker {
val config = sharedServer.brokerConfig
val time = sharedServer.time
def metrics = sharedServer.metrics
// Get raftManager from SharedServer. It will be initialized during startup.
def raftManager: KafkaRaftManager[ApiMessageAndVersion] = sharedServer.raftManager
override def brokerState: BrokerState = Option(lifecycleManager).
flatMap(m => Some(m.state)).getOrElse(BrokerState.NOT_RUNNING)
import kafka.server.Server._
private val logContext: LogContext = new LogContext(s"[BrokerServer id=${config.nodeId}] ")
this.logIdent = logContext.logPrefix
@volatile var lifecycleManager: BrokerLifecycleManager = _
private val isShuttingDown = new AtomicBoolean(false)
val lock = new ReentrantLock()
val awaitShutdownCond = lock.newCondition()
var status: ProcessStatus = SHUTDOWN
@volatile var dataPlaneRequestProcessor: KafkaApis = _
var controlPlaneRequestProcessor: KafkaApis = _
var authorizer: Option[Authorizer] = None
@volatile var socketServer: SocketServer = _
var dataPlaneRequestHandlerPool: KafkaRequestHandlerPool = _
var logDirFailureChannel: LogDirFailureChannel = _
var logManager: LogManager = _
var remoteLogManagerOpt: Option[RemoteLogManager] = None
var tokenManager: DelegationTokenManager = _
var dynamicConfigHandlers: Map[String, ConfigHandler] = _
@volatile private[this] var _replicaManager: ReplicaManager = _
var credentialProvider: CredentialProvider = _
var tokenCache: DelegationTokenCache = _
@volatile var groupCoordinator: GroupCoordinator = _
var transactionCoordinator: TransactionCoordinator = _
var clientToControllerChannelManager: BrokerToControllerChannelManager = _
var forwardingManager: ForwardingManager = _
var alterPartitionManager: AlterPartitionManager = _
var autoTopicCreationManager: AutoTopicCreationManager = _
var kafkaScheduler: KafkaScheduler = _
@volatile var metadataCache: KRaftMetadataCache = _
var quotaManagers: QuotaFactory.QuotaManagers = _
var clientQuotaMetadataManager: ClientQuotaMetadataManager = _
@volatile var brokerTopicStats: BrokerTopicStats = _
val clusterId: String = sharedServer.metaProps.clusterId
var brokerMetadataPublisher: BrokerMetadataPublisher = _
val brokerFeatures: BrokerFeatures = BrokerFeatures.createDefault()
def kafkaYammerMetrics: KafkaYammerMetrics = KafkaYammerMetrics.INSTANCE
val metadataPublishers: util.List[MetadataPublisher] = new util.ArrayList[MetadataPublisher]()
private def maybeChangeStatus(from: ProcessStatus, to: ProcessStatus): Boolean = {
lock.lock()
try {
if (status != from) return false
info(s"Transition from $status to $to")
status = to
if (to == SHUTTING_DOWN) {
isShuttingDown.set(true)
} else if (to == SHUTDOWN) {
isShuttingDown.set(false)
awaitShutdownCond.signalAll()
}
} finally {
lock.unlock()
}
true
}
def replicaManager: ReplicaManager = _replicaManager
override def startup(): Unit = {
if (!maybeChangeStatus(SHUTDOWN, STARTING)) return
val startupDeadline = Deadline.fromDelay(time, config.serverMaxStartupTimeMs, TimeUnit.MILLISECONDS)
try {
sharedServer.startForBroker()
info("Starting broker")
config.dynamicConfig.initialize(zkClientOpt = None)
lifecycleManager = new BrokerLifecycleManager(config,
time,
s"broker-${config.nodeId}-",
isZkBroker = false)
/* start scheduler */
kafkaScheduler = new KafkaScheduler(config.backgroundThreads)
kafkaScheduler.startup()
/* register broker metrics */
brokerTopicStats = new BrokerTopicStats(java.util.Optional.of(config))
quotaManagers = QuotaFactory.instantiate(config, metrics, time, s"broker-${config.nodeId}-")
logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size)
metadataCache = MetadataCache.kRaftMetadataCache(config.nodeId)
// Create log manager, but don't start it because we need to delay any potential unclean shutdown log recovery
// until we catch up on the metadata log and have up-to-date topic and broker configs.
logManager = LogManager(config, initialOfflineDirs, metadataCache, kafkaScheduler, time,
brokerTopicStats, logDirFailureChannel, keepPartitionMetadataFile = true)
remoteLogManagerOpt = createRemoteLogManager()
// Enable delegation token cache for all SCRAM mechanisms to simplify dynamic update.
// This keeps the cache up-to-date if new SCRAM mechanisms are enabled dynamically.
tokenCache = new DelegationTokenCache(ScramMechanism.mechanismNames)
credentialProvider = new CredentialProvider(ScramMechanism.mechanismNames, tokenCache)
val voterConnections = FutureUtils.waitWithLogging(logger.underlying, logIdent,
"controller quorum voters future",
sharedServer.controllerQuorumVotersFuture,
startupDeadline, time)
val controllerNodes = RaftConfig.voterConnectionsToNodes(voterConnections).asScala
val controllerNodeProvider = RaftControllerNodeProvider(raftManager, config, controllerNodes)
clientToControllerChannelManager = BrokerToControllerChannelManager(
controllerNodeProvider,
time,
metrics,
config,
channelName = "forwarding",
s"broker-${config.nodeId}-",
retryTimeoutMs = 60000
)
clientToControllerChannelManager.start()
forwardingManager = new ForwardingManagerImpl(clientToControllerChannelManager)
val apiVersionManager = ApiVersionManager(
ListenerType.BROKER,
config,
Some(forwardingManager),
brokerFeatures,
metadataCache
)
// Create and start the socket server acceptor threads so that the bound port is known.
// Delay starting processors until the end of the initialization sequence to ensure
// that credentials have been loaded before processing authentications.
socketServer = new SocketServer(config, metrics, time, credentialProvider, apiVersionManager)
clientQuotaMetadataManager = new ClientQuotaMetadataManager(quotaManagers, socketServer.connectionQuotas)
alterPartitionManager = AlterPartitionManager(
config,
metadataCache,
scheduler = kafkaScheduler,
controllerNodeProvider,
time = time,
metrics,
s"broker-${config.nodeId}-",
brokerEpochSupplier = () => lifecycleManager.brokerEpoch
)
alterPartitionManager.start()
val addPartitionsLogContext = new LogContext(s"[AddPartitionsToTxnManager broker=${config.brokerId}]")
val addPartitionsToTxnNetworkClient: NetworkClient = NetworkUtils.buildNetworkClient("AddPartitionsManager", config, metrics, time, addPartitionsLogContext)
val addPartitionsToTxnManager: AddPartitionsToTxnManager = new AddPartitionsToTxnManager(config, addPartitionsToTxnNetworkClient, time)
this._replicaManager = new ReplicaManager(
config = config,
metrics = metrics,
time = time,
scheduler = kafkaScheduler,
logManager = logManager,
remoteLogManager = remoteLogManagerOpt,
quotaManagers = quotaManagers,
metadataCache = metadataCache,
logDirFailureChannel = logDirFailureChannel,
alterPartitionManager = alterPartitionManager,
brokerTopicStats = brokerTopicStats,
isShuttingDown = isShuttingDown,
zkClient = None,
threadNamePrefix = None, // The ReplicaManager only runs on the broker, and already includes the ID in thread names.
delayedRemoteFetchPurgatoryParam = None,
brokerEpochSupplier = () => lifecycleManager.brokerEpoch,
addPartitionsToTxnManager = Some(addPartitionsToTxnManager)
)
/* start token manager */
if (config.tokenAuthEnabled) {
throw new UnsupportedOperationException("Delegation tokens are not supported")
}
tokenManager = new DelegationTokenManager(config, tokenCache, time , null)
tokenManager.startup() // does nothing, we just need a token manager in order to compile right now...
groupCoordinator = createGroupCoordinator()
val producerIdManagerSupplier = () => ProducerIdManager.rpc(
config.brokerId,
time,
brokerEpochSupplier = () => lifecycleManager.brokerEpoch,
clientToControllerChannelManager
)
// Create transaction coordinator, but don't start it until we've started replica manager.
// Hardcode Time.SYSTEM for now as some Streams tests fail otherwise, it would be good to fix the underlying issue
transactionCoordinator = TransactionCoordinator(config, replicaManager,
new KafkaScheduler(1, true, "transaction-log-manager-"),
producerIdManagerSupplier, metrics, metadataCache, Time.SYSTEM)
autoTopicCreationManager = new DefaultAutoTopicCreationManager(
config, Some(clientToControllerChannelManager), None, None,
groupCoordinator, transactionCoordinator)
dynamicConfigHandlers = Map[String, ConfigHandler](
ConfigType.Topic -> new TopicConfigHandler(logManager, config, quotaManagers, None),
ConfigType.Broker -> new BrokerConfigHandler(config, quotaManagers))
val networkListeners = new ListenerCollection()
config.effectiveAdvertisedListeners.foreach { ep =>
networkListeners.add(new Listener().
setHost(if (Utils.isBlank(ep.host)) InetAddress.getLocalHost.getCanonicalHostName else ep.host).
setName(ep.listenerName.value()).
setPort(if (ep.port == 0) socketServer.boundPort(ep.listenerName) else ep.port).
setSecurityProtocol(ep.securityProtocol.id))
}
val featuresRemapped = brokerFeatures.supportedFeatures.features().asScala.map {
case (k: String, v: SupportedVersionRange) =>
k -> VersionRange.of(v.min, v.max)
}.asJava
val brokerLifecycleChannelManager = BrokerToControllerChannelManager(
controllerNodeProvider,
time,
metrics,
config,
"heartbeat",
s"broker-${config.nodeId}-",
config.brokerSessionTimeoutMs / 2 // KAFKA-14392
)
lifecycleManager.start(
() => sharedServer.loader.lastAppliedOffset(),
brokerLifecycleChannelManager,
sharedServer.metaProps.clusterId,
networkListeners,
featuresRemapped
)
// If the BrokerLifecycleManager's initial catch-up future fails, it means we timed out
// or are shutting down before we could catch up. Therefore, also fail the firstPublishFuture.
lifecycleManager.initialCatchUpFuture.whenComplete((_, e) => {
if (e != null) brokerMetadataPublisher.firstPublishFuture.completeExceptionally(e)
})
val endpoints = new util.ArrayList[Endpoint](networkListeners.size())
var interBrokerListener: Endpoint = null
networkListeners.iterator().forEachRemaining(listener => {
val endPoint = new Endpoint(listener.name(),
SecurityProtocol.forId(listener.securityProtocol()),
listener.host(), listener.port())
endpoints.add(endPoint)
if (listener.name().equals(config.interBrokerListenerName.value())) {
interBrokerListener = endPoint
}
})
if (interBrokerListener == null) {
throw new RuntimeException("Unable to find inter-broker listener " +
config.interBrokerListenerName.value() + ". Found listener(s): " +
endpoints.asScala.map(ep => ep.listenerName().orElse("(none)")).mkString(", "))
}
// Create and initialize an authorizer if one is configured.
authorizer = config.createNewAuthorizer()
authorizer.foreach(_.configure(config.originals))
val fetchManager = new FetchManager(Time.SYSTEM,
new FetchSessionCache(config.maxIncrementalFetchSessionCacheSlots,
KafkaServer.MIN_INCREMENTAL_FETCH_SESSION_EVICTION_MS))
// Create the request processor objects.
val raftSupport = RaftSupport(forwardingManager, metadataCache)
dataPlaneRequestProcessor = new KafkaApis(
requestChannel = socketServer.dataPlaneRequestChannel,
metadataSupport = raftSupport,
replicaManager = replicaManager,
groupCoordinator = groupCoordinator,
txnCoordinator = transactionCoordinator,
autoTopicCreationManager = autoTopicCreationManager,
brokerId = config.nodeId,
config = config,
configRepository = metadataCache,
metadataCache = metadataCache,
metrics = metrics,
authorizer = authorizer,
quotas = quotaManagers,
fetchManager = fetchManager,
brokerTopicStats = brokerTopicStats,
clusterId = clusterId,
time = time,
tokenManager = tokenManager,
apiVersionManager = apiVersionManager)
dataPlaneRequestHandlerPool = new KafkaRequestHandlerPool(config.nodeId,
socketServer.dataPlaneRequestChannel, dataPlaneRequestProcessor, time,
config.numIoThreads, s"${DataPlaneAcceptor.MetricPrefix}RequestHandlerAvgIdlePercent",
DataPlaneAcceptor.ThreadPrefix)
brokerMetadataPublisher = new BrokerMetadataPublisher(config,
metadataCache,
logManager,
replicaManager,
groupCoordinator,
transactionCoordinator,
new DynamicConfigPublisher(
config,
sharedServer.metadataPublishingFaultHandler,
dynamicConfigHandlers.toMap,
"broker"),
new DynamicClientQuotaPublisher(
config,
sharedServer.metadataPublishingFaultHandler,
"broker",
clientQuotaMetadataManager),
new ScramPublisher(
config,
sharedServer.metadataPublishingFaultHandler,
"broker",
credentialProvider),
authorizer,
sharedServer.initialBrokerMetadataLoadFaultHandler,
sharedServer.metadataPublishingFaultHandler)
metadataPublishers.add(brokerMetadataPublisher)
// Register parts of the broker that can be reconfigured via dynamic configs. This needs to
// be done before we publish the dynamic configs, so that we don't miss anything.
config.dynamicConfig.addReconfigurables(this)
// Install all the metadata publishers.
FutureUtils.waitWithLogging(logger.underlying, logIdent,
"the broker metadata publishers to be installed",
sharedServer.loader.installPublishers(metadataPublishers), startupDeadline, time)
// Wait for this broker to contact the quorum, and for the active controller to acknowledge
// us as caught up. It will do this by returning a heartbeat response with isCaughtUp set to
// true. The BrokerLifecycleManager tracks this.
FutureUtils.waitWithLogging(logger.underlying, logIdent,
"the controller to acknowledge that we are caught up",
lifecycleManager.initialCatchUpFuture, startupDeadline, time)
// Wait for the first metadata update to be published. Metadata updates are not published
// until we read at least up to the high water mark of the cluster metadata partition.
// Usually, we publish the initial metadata before lifecycleManager.initialCatchUpFuture
// is completed, so this check is not necessary. But this is a simple check to make
// completely sure.
FutureUtils.waitWithLogging(logger.underlying, logIdent,
"the initial broker metadata update to be published",
brokerMetadataPublisher.firstPublishFuture , startupDeadline, time)
// Now that we have loaded some metadata, we can log a reasonably up-to-date broker
// configuration. Keep in mind that KafkaConfig.originals is a mutable field that gets set
// by the dynamic configuration publisher. Ironically, KafkaConfig.originals does not
// contain the original configuration values.
new KafkaConfig(config.originals(), true)
// Start RemoteLogManager before broker start serving the requests.
remoteLogManagerOpt.foreach(rlm => {
val listenerName = config.remoteLogManagerConfig.remoteLogMetadataManagerListenerName()
if (listenerName != null) {
val endpoint = endpoints.stream.filter(e => e.listenerName.equals(ListenerName.normalised(listenerName)))
.findFirst()
.orElseThrow(() => new ConfigException(RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_LISTENER_NAME_PROP +
" should be set as a listener name within valid broker listener name list."))
rlm.onEndPointCreated(EndPoint.fromJava(endpoint))
}
rlm.startup()
})
// If we are using a ClusterMetadataAuthorizer which stores its ACLs in the metadata log,
// notify it that the loading process is complete.
authorizer match {
case Some(clusterMetadataAuthorizer: ClusterMetadataAuthorizer) =>
clusterMetadataAuthorizer.completeInitialLoad()
case _ => // nothing to do
}
// We're now ready to unfence the broker. This also allows this broker to transition
// from RECOVERY state to RUNNING state, once the controller unfences the broker.
FutureUtils.waitWithLogging(logger.underlying, logIdent,
"the broker to be unfenced",
lifecycleManager.setReadyToUnfence(), startupDeadline, time)
// Enable inbound TCP connections. Each endpoint will be started only once its matching
// authorizer future is completed.
val endpointReadyFutures = {
val builder = new EndpointReadyFutures.Builder()
builder.build(authorizer.asJava,
new KafkaAuthorizerServerInfo(
new ClusterResource(clusterId),
config.nodeId,
endpoints,
interBrokerListener,
config.earlyStartListeners.map(_.value()).asJava))
}
val authorizerFutures = endpointReadyFutures.futures().asScala.toMap
val enableRequestProcessingFuture = socketServer.enableRequestProcessing(authorizerFutures)
// 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",
enableRequestProcessingFuture, startupDeadline, time)
maybeChangeStatus(STARTING, STARTED)
} catch {
case e: Throwable =>
maybeChangeStatus(STARTING, STARTED)
fatal("Fatal error during broker startup. Prepare to shutdown", e)
shutdown()
throw if (e.isInstanceOf[ExecutionException]) e.getCause else e
}
}
private def createGroupCoordinator(): GroupCoordinator = {
// Create group coordinator, but don't start it until we've started replica manager.
// Hardcode Time.SYSTEM for now as some Streams tests fail otherwise, it would be good
// to fix the underlying issue.
if (config.isNewGroupCoordinatorEnabled) {
val time = Time.SYSTEM
val serde = new RecordSerde
val groupCoordinatorConfig = new GroupCoordinatorConfig(
config.groupCoordinatorNumThreads,
config.consumerGroupSessionTimeoutMs,
config.consumerGroupHeartbeatIntervalMs,
config.consumerGroupMaxSize,
config.consumerGroupAssignors,
config.offsetsTopicSegmentBytes,
config.offsetMetadataMaxSize,
config.groupMaxSize,
config.groupInitialRebalanceDelay,
GroupCoordinatorConfig.GENERIC_GROUP_NEW_MEMBER_JOIN_TIMEOUT_MS,
config.groupMinSessionTimeoutMs,
config.groupMaxSessionTimeoutMs
)
val timer = new SystemTimerReaper(
"group-coordinator-reaper",
new SystemTimer("group-coordinator")
)
val loader = new CoordinatorLoaderImpl[group.Record](
replicaManager,
serde,
config.offsetsLoadBufferSize
)
val writer = new CoordinatorPartitionWriter[group.Record](
replicaManager,
serde,
config.offsetsTopicCompressionType,
time
)
new GroupCoordinatorService.Builder(config.brokerId, groupCoordinatorConfig)
.withTime(time)
.withTimer(timer)
.withLoader(loader)
.withWriter(writer)
.build()
} else {
GroupCoordinatorAdapter(
config,
replicaManager,
Time.SYSTEM,
metrics
)
}
}
protected def createRemoteLogManager(): Option[RemoteLogManager] = {
if (config.remoteLogManagerConfig.enableRemoteStorageSystem()) {
if (config.logDirs.size > 1) {
throw new KafkaException("Tiered storage is not supported with multiple log dirs.");
}
Some(new RemoteLogManager(config.remoteLogManagerConfig, config.brokerId, config.logDirs.head, clusterId, time,
(tp: TopicPartition) => logManager.getLog(tp).asJava, brokerTopicStats));
} else {
None
}
}
override def shutdown(): Unit = {
if (!maybeChangeStatus(STARTED, SHUTTING_DOWN)) return
try {
info("shutting down")
if (config.controlledShutdownEnable) {
if (replicaManager != null)
replicaManager.beginControlledShutdown()
lifecycleManager.beginControlledShutdown()
try {
lifecycleManager.controlledShutdownFuture.get(5L, TimeUnit.MINUTES)
} catch {
case _: TimeoutException =>
error("Timed out waiting for the controller to approve controlled shutdown")
case e: Throwable =>
error("Got unexpected exception waiting for controlled shutdown future", e)
}
}
lifecycleManager.beginShutdown()
// Stop socket server to stop accepting any more connections and requests.
// Socket server will be shutdown towards the end of the sequence.
if (socketServer != null) {
CoreUtils.swallow(socketServer.stopProcessingRequests(), this)
}
metadataPublishers.forEach(p => sharedServer.loader.removeAndClosePublisher(p).get())
metadataPublishers.clear()
if (dataPlaneRequestHandlerPool != null)
CoreUtils.swallow(dataPlaneRequestHandlerPool.shutdown(), this)
if (dataPlaneRequestProcessor != null)
CoreUtils.swallow(dataPlaneRequestProcessor.close(), this)
if (controlPlaneRequestProcessor != null)
CoreUtils.swallow(controlPlaneRequestProcessor.close(), this)
CoreUtils.swallow(authorizer.foreach(_.close()), this)
/**
* We must shutdown the scheduler early because otherwise, the scheduler could touch other
* resources that might have been shutdown and cause exceptions.
* For example, if we didn't shutdown the scheduler first, when LogManager was closing
* partitions one by one, the scheduler might concurrently delete old segments due to
* retention. However, the old segments could have been closed by the LogManager, which would
* cause an IOException and subsequently mark logdir as offline. As a result, the broker would
* not flush the remaining partitions or write the clean shutdown marker. Ultimately, the
* broker would have to take hours to recover the log during restart.
*/
if (kafkaScheduler != null)
CoreUtils.swallow(kafkaScheduler.shutdown(), this)
if (transactionCoordinator != null)
CoreUtils.swallow(transactionCoordinator.shutdown(), this)
if (groupCoordinator != null)
CoreUtils.swallow(groupCoordinator.shutdown(), this)
if (tokenManager != null)
CoreUtils.swallow(tokenManager.shutdown(), this)
if (replicaManager != null)
CoreUtils.swallow(replicaManager.shutdown(), this)
if (alterPartitionManager != null)
CoreUtils.swallow(alterPartitionManager.shutdown(), this)
if (clientToControllerChannelManager != null)
CoreUtils.swallow(clientToControllerChannelManager.shutdown(), this)
if (logManager != null)
CoreUtils.swallow(logManager.shutdown(), this)
// Close remote log manager to give a chance to any of its underlying clients
// (especially in RemoteStorageManager and RemoteLogMetadataManager) to close gracefully.
CoreUtils.swallow(remoteLogManagerOpt.foreach(_.close()), this)
if (quotaManagers != null)
CoreUtils.swallow(quotaManagers.shutdown(), this)
if (socketServer != null)
CoreUtils.swallow(socketServer.shutdown(), this)
if (brokerTopicStats != null)
CoreUtils.swallow(brokerTopicStats.close(), this)
isShuttingDown.set(false)
CoreUtils.swallow(lifecycleManager.close(), this)
CoreUtils.swallow(config.dynamicConfig.clear(), this)
sharedServer.stopForBroker()
info("shut down completed")
} catch {
case e: Throwable =>
fatal("Fatal error during broker shutdown.", e)
throw e
} finally {
maybeChangeStatus(SHUTTING_DOWN, SHUTDOWN)
}
}
override def awaitShutdown(): Unit = {
lock.lock()
try {
while (true) {
if (status == SHUTDOWN) return
awaitShutdownCond.awaitUninterruptibly()
}
} finally {
lock.unlock()
}
}
override def boundPort(listenerName: ListenerName): Int = socketServer.boundPort(listenerName)
}