forked from apache/kafka
-
Notifications
You must be signed in to change notification settings - Fork 0
/
LogManager.scala
executable file
·1669 lines (1499 loc) · 70.8 KB
/
LogManager.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
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/**
* 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.log
import java.io._
import java.nio.file.{Files, NoSuchFileException}
import java.util.concurrent._
import java.util.concurrent.atomic.AtomicInteger
import kafka.server.checkpoints.OffsetCheckpointFile
import kafka.server.metadata.ConfigRepository
import kafka.server._
import kafka.server.metadata.BrokerMetadataPublisher.info
import kafka.utils._
import org.apache.kafka.common.{DirectoryId, KafkaException, TopicPartition, Uuid}
import org.apache.kafka.common.utils.{KafkaThread, Time, Utils}
import org.apache.kafka.common.errors.{InconsistentTopicIdException, KafkaStorageException, LogDirNotFoundException}
import scala.jdk.CollectionConverters._
import scala.collection._
import scala.collection.mutable.ArrayBuffer
import scala.util.{Failure, Success, Try}
import kafka.utils.Implicits._
import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.requests.{AbstractControlRequest, LeaderAndIsrRequest}
import org.apache.kafka.image.TopicsImage
import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, PropertiesUtils}
import java.util.{OptionalLong, Properties}
import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.storage.internals.log.LogConfig.MessageFormatVersion
import org.apache.kafka.server.metrics.KafkaMetricsGroup
import org.apache.kafka.server.util.Scheduler
import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig, RemoteIndexCache}
import org.apache.kafka.storage.internals.checkpoint.CleanShutdownFileHandler
import java.util
import scala.annotation.nowarn
/**
* The entry point to the kafka log management subsystem. The log manager is responsible for log creation, retrieval, and cleaning.
* All read and write operations are delegated to the individual log instances.
*
* The log manager maintains logs in one or more directories. New logs are created in the data directory
* with the fewest logs. No attempt is made to move partitions after the fact or balance based on
* size or I/O rate.
*
* A background thread handles log retention by periodically truncating excess log segments.
*/
@threadsafe
class LogManager(logDirs: Seq[File],
initialOfflineDirs: Seq[File],
configRepository: ConfigRepository,
val initialDefaultConfig: LogConfig,
val cleanerConfig: CleanerConfig,
recoveryThreadsPerDataDir: Int,
val flushCheckMs: Long,
val flushRecoveryOffsetCheckpointMs: Long,
val flushStartOffsetCheckpointMs: Long,
val retentionCheckMs: Long,
val maxTransactionTimeoutMs: Int,
val producerStateManagerConfig: ProducerStateManagerConfig,
val producerIdExpirationCheckIntervalMs: Int,
interBrokerProtocolVersion: MetadataVersion,
scheduler: Scheduler,
brokerTopicStats: BrokerTopicStats,
logDirFailureChannel: LogDirFailureChannel,
time: Time,
val keepPartitionMetadataFile: Boolean,
remoteStorageSystemEnable: Boolean,
val initialTaskDelayMs: Long) extends Logging {
import LogManager._
private val metricsGroup = new KafkaMetricsGroup(this.getClass)
private val logCreationOrDeletionLock = new Object
private val currentLogs = new Pool[TopicPartition, UnifiedLog]()
// Future logs are put in the directory with "-future" suffix. Future log is created when user wants to move replica
// from one log directory to another log directory on the same broker. The directory of the future log will be renamed
// to replace the current log of the partition after the future log catches up with the current log
private val futureLogs = new Pool[TopicPartition, UnifiedLog]()
// Each element in the queue contains the log object to be deleted and the time it is scheduled for deletion.
private val logsToBeDeleted = new LinkedBlockingQueue[(UnifiedLog, Long)]()
// Map of stray partition to stray log. This holds all stray logs detected on the broker.
// Visible for testing
private val strayLogs = new Pool[TopicPartition, UnifiedLog]()
private val _liveLogDirs: ConcurrentLinkedQueue[File] = createAndValidateLogDirs(logDirs, initialOfflineDirs)
@volatile private var _currentDefaultConfig = initialDefaultConfig
@volatile private var numRecoveryThreadsPerDataDir = recoveryThreadsPerDataDir
// This map contains all partitions whose logs are getting loaded and initialized. If log configuration
// of these partitions get updated at the same time, the corresponding entry in this map is set to "true",
// which triggers a config reload after initialization is finished (to get the latest config value).
// See KAFKA-8813 for more detail on the race condition
// Visible for testing
private[log] val partitionsInitializing = new ConcurrentHashMap[TopicPartition, Boolean]().asScala
def reconfigureDefaultLogConfig(logConfig: LogConfig): Unit = {
this._currentDefaultConfig = logConfig
}
def currentDefaultConfig: LogConfig = _currentDefaultConfig
def liveLogDirs: Seq[File] = {
if (_liveLogDirs.size == logDirs.size)
logDirs
else
_liveLogDirs.asScala.toBuffer
}
private val dirLocks = lockLogDirs(liveLogDirs)
private val directoryIds: mutable.Map[String, Uuid] = loadDirectoryIds(liveLogDirs)
def directoryIdsSet: Predef.Set[Uuid] = directoryIds.values.toSet
@volatile private var recoveryPointCheckpoints = liveLogDirs.map(dir =>
(dir, new OffsetCheckpointFile(new File(dir, RecoveryPointCheckpointFile), logDirFailureChannel))).toMap
@volatile private var logStartOffsetCheckpoints = liveLogDirs.map(dir =>
(dir, new OffsetCheckpointFile(new File(dir, LogStartOffsetCheckpointFile), logDirFailureChannel))).toMap
private val preferredLogDirs = new ConcurrentHashMap[TopicPartition, String]()
def hasOfflineLogDirs(): Boolean = offlineLogDirs.nonEmpty
def onlineLogDirId(uuid: Uuid): Boolean = directoryIds.exists(_._2 == uuid)
private def offlineLogDirs: Iterable[File] = {
val logDirsSet = mutable.Set[File]() ++= logDirs
_liveLogDirs.forEach(dir => logDirsSet -= dir)
logDirsSet
}
// A map that stores hadCleanShutdown flag for each log dir.
private val hadCleanShutdownFlags = new ConcurrentHashMap[String, Boolean]()
// A map that tells whether all logs in a log dir had been loaded or not at startup time.
private val loadLogsCompletedFlags = new ConcurrentHashMap[String, Boolean]()
@volatile private var _cleaner: LogCleaner = _
private[kafka] def cleaner: LogCleaner = _cleaner
metricsGroup.newGauge("OfflineLogDirectoryCount", () => offlineLogDirs.size)
for (dir <- logDirs) {
metricsGroup.newGauge("LogDirectoryOffline",
() => if (_liveLogDirs.contains(dir)) 0 else 1,
Map("logDirectory" -> dir.getAbsolutePath).asJava)
}
/**
* Create and check validity of the given directories that are not in the given offline directories, specifically:
* <ol>
* <li> Ensure that there are no duplicates in the directory list
* <li> Create each directory if it doesn't exist
* <li> Check that each path is a readable directory
* </ol>
*/
private def createAndValidateLogDirs(dirs: Seq[File], initialOfflineDirs: Seq[File]): ConcurrentLinkedQueue[File] = {
val liveLogDirs = new ConcurrentLinkedQueue[File]()
val canonicalPaths = mutable.HashSet.empty[String]
for (dir <- dirs) {
try {
if (initialOfflineDirs.contains(dir))
throw new IOException(s"Failed to load ${dir.getAbsolutePath} during broker startup")
if (!dir.exists) {
info(s"Log directory ${dir.getAbsolutePath} not found, creating it.")
val created = dir.mkdirs()
if (!created)
throw new IOException(s"Failed to create data directory ${dir.getAbsolutePath}")
Utils.flushDir(dir.toPath.toAbsolutePath.normalize.getParent)
}
if (!dir.isDirectory || !dir.canRead)
throw new IOException(s"${dir.getAbsolutePath} is not a readable log directory.")
// getCanonicalPath() throws IOException if a file system query fails or if the path is invalid (e.g. contains
// the Nul character). Since there's no easy way to distinguish between the two cases, we treat them the same
// and mark the log directory as offline.
if (!canonicalPaths.add(dir.getCanonicalPath))
throw new KafkaException(s"Duplicate log directory found: ${dirs.mkString(", ")}")
liveLogDirs.add(dir)
} catch {
case e: IOException =>
logDirFailureChannel.maybeAddOfflineLogDir(dir.getAbsolutePath, s"Failed to create or validate data directory ${dir.getAbsolutePath}", e)
}
}
if (liveLogDirs.isEmpty) {
fatal(s"Shutdown broker because none of the specified log dirs from ${dirs.mkString(", ")} can be created or validated")
Exit.halt(1)
}
liveLogDirs
}
def resizeRecoveryThreadPool(newSize: Int): Unit = {
info(s"Resizing recovery thread pool size for each data dir from $numRecoveryThreadsPerDataDir to $newSize")
numRecoveryThreadsPerDataDir = newSize
}
/**
* The log directory failure handler. It will stop log cleaning in that directory.
*
* @param dir the absolute path of the log directory
*/
def handleLogDirFailure(dir: String): Unit = {
warn(s"Stopping serving logs in dir $dir")
logCreationOrDeletionLock synchronized {
_liveLogDirs.remove(new File(dir))
directoryIds.remove(dir)
if (_liveLogDirs.isEmpty) {
fatal(s"Shutdown broker because all log dirs in ${logDirs.mkString(", ")} have failed")
Exit.halt(1)
}
recoveryPointCheckpoints = recoveryPointCheckpoints.filter { case (file, _) => file.getAbsolutePath != dir }
logStartOffsetCheckpoints = logStartOffsetCheckpoints.filter { case (file, _) => file.getAbsolutePath != dir }
if (cleaner != null)
cleaner.handleLogDirFailure(dir)
def removeOfflineLogs(logs: Pool[TopicPartition, UnifiedLog]): Iterable[TopicPartition] = {
val offlineTopicPartitions: Iterable[TopicPartition] = logs.collect {
case (tp, log) if log.parentDir == dir => tp
}
offlineTopicPartitions.foreach { topicPartition => {
val removedLog = removeLogAndMetrics(logs, topicPartition)
removedLog.foreach {
log => log.closeHandlers()
}
}}
offlineTopicPartitions
}
val offlineCurrentTopicPartitions = removeOfflineLogs(currentLogs)
val offlineFutureTopicPartitions = removeOfflineLogs(futureLogs)
warn(s"Logs for partitions ${offlineCurrentTopicPartitions.mkString(",")} are offline and " +
s"logs for future partitions ${offlineFutureTopicPartitions.mkString(",")} are offline due to failure on log directory $dir")
dirLocks.filter(_.file.getParent == dir).foreach(dir => CoreUtils.swallow(dir.destroy(), this))
}
}
/**
* Lock all the given directories
*/
private def lockLogDirs(dirs: Seq[File]): Seq[FileLock] = {
dirs.flatMap { dir =>
try {
val lock = new FileLock(new File(dir, LockFileName))
if (!lock.tryLock())
throw new KafkaException("Failed to acquire lock on file .lock in " + lock.file.getParent +
". A Kafka instance in another process or thread is using this directory.")
Some(lock)
} catch {
case e: IOException =>
logDirFailureChannel.maybeAddOfflineLogDir(dir.getAbsolutePath, s"Disk error while locking directory $dir", e)
None
}
}
}
/**
* Retrieves the Uuid for the directory, given its absolute path.
*/
def directoryId(dir: String): Option[Uuid] = directoryIds.get(dir)
def directoryPath(uuid: Uuid): Option[String] = directoryIds.find(_._2 == uuid).map(_._1)
/**
* Determine directory ID for each directory with a meta.properties.
* If meta.properties does not include a directory ID, one is generated and persisted back to meta.properties.
* Directories without a meta.properties don't get a directory ID assigned.
*/
private def loadDirectoryIds(logDirs: Seq[File]): mutable.Map[String, Uuid] = {
val result = mutable.HashMap[String, Uuid]()
logDirs.foreach(logDir => {
try {
val props = PropertiesUtils.readPropertiesFile(
new File(logDir, MetaPropertiesEnsemble.META_PROPERTIES_NAME).getAbsolutePath)
val metaProps = new MetaProperties.Builder(props).build()
metaProps.directoryId().ifPresent(directoryId => {
result += (logDir.getAbsolutePath -> directoryId)
})
} catch {
case _: NoSuchFileException =>
info(s"No meta.properties file found in $logDir.")
case e: IOException =>
logDirFailureChannel.maybeAddOfflineLogDir(logDir.getAbsolutePath, s"Disk error while loading ID $logDir", e)
}
})
result
}
private def addLogToBeDeleted(log: UnifiedLog): Unit = {
this.logsToBeDeleted.add((log, time.milliseconds()))
}
def addStrayLog(strayPartition: TopicPartition, strayLog: UnifiedLog): Unit = {
this.strayLogs.put(strayPartition, strayLog)
}
// Only for testing
private[log] def hasLogsToBeDeleted: Boolean = !logsToBeDeleted.isEmpty
private[log] def loadLog(logDir: File,
hadCleanShutdown: Boolean,
recoveryPoints: Map[TopicPartition, Long],
logStartOffsets: Map[TopicPartition, Long],
defaultConfig: LogConfig,
topicConfigOverrides: Map[String, LogConfig],
numRemainingSegments: ConcurrentMap[String, Int],
isStray: UnifiedLog => Boolean): UnifiedLog = {
val topicPartition = UnifiedLog.parseTopicPartitionName(logDir)
val config = topicConfigOverrides.getOrElse(topicPartition.topic, defaultConfig)
val logRecoveryPoint = recoveryPoints.getOrElse(topicPartition, 0L)
val logStartOffset = logStartOffsets.getOrElse(topicPartition, 0L)
val log = UnifiedLog(
dir = logDir,
config = config,
logStartOffset = logStartOffset,
recoveryPoint = logRecoveryPoint,
maxTransactionTimeoutMs = maxTransactionTimeoutMs,
producerStateManagerConfig = producerStateManagerConfig,
producerIdExpirationCheckIntervalMs = producerIdExpirationCheckIntervalMs,
scheduler = scheduler,
time = time,
brokerTopicStats = brokerTopicStats,
logDirFailureChannel = logDirFailureChannel,
lastShutdownClean = hadCleanShutdown,
topicId = None,
keepPartitionMetadataFile = keepPartitionMetadataFile,
numRemainingSegments = numRemainingSegments,
remoteStorageSystemEnable = remoteStorageSystemEnable)
if (logDir.getName.endsWith(UnifiedLog.DeleteDirSuffix)) {
addLogToBeDeleted(log)
} else if (logDir.getName.endsWith(UnifiedLog.StrayDirSuffix)) {
addStrayLog(topicPartition, log)
warn(s"Loaded stray log: $logDir")
} else if (isStray(log)) {
// Unlike Zookeeper mode, which tracks pending topic deletions under a ZNode, KRaft is unable to prevent a topic from being recreated before every replica has been deleted.
// A KRaft broker with an offline directory may be unable to detect it still holds a to-be-deleted replica,
// and can create a conflicting topic partition for a new incarnation of the topic in one of the remaining online directories.
// So upon a restart in which the offline directory is back online we need to clean up the old replica directory.
log.renameDir(UnifiedLog.logStrayDirName(log.topicPartition), shouldReinitialize = false)
addStrayLog(log.topicPartition, log)
warn(s"Log in ${logDir.getAbsolutePath} marked stray and renamed to ${log.dir.getAbsolutePath}")
} else {
val previous = {
if (log.isFuture)
this.futureLogs.put(topicPartition, log)
else
this.currentLogs.put(topicPartition, log)
}
if (previous != null) {
if (log.isFuture)
throw new IllegalStateException(s"Duplicate log directories found: ${log.dir.getAbsolutePath}, ${previous.dir.getAbsolutePath}")
else
throw new IllegalStateException(s"Duplicate log directories for $topicPartition are found in both ${log.dir.getAbsolutePath} " +
s"and ${previous.dir.getAbsolutePath}. It is likely because log directory failure happened while broker was " +
s"replacing current replica with future replica. Recover broker from this failure by manually deleting one of the two directories " +
s"for this partition. It is recommended to delete the partition in the log directory that is known to have failed recently.")
}
}
log
}
// factory class for naming the log recovery threads used in metrics
private class LogRecoveryThreadFactory(val dirPath: String) extends ThreadFactory {
val threadNum = new AtomicInteger(0)
override def newThread(runnable: Runnable): Thread = {
KafkaThread.nonDaemon(logRecoveryThreadName(dirPath, threadNum.getAndIncrement()), runnable)
}
}
// create a unique log recovery thread name for each log dir as the format: prefix-dirPath-threadNum, ex: "log-recovery-/tmp/kafkaLogs-0"
private def logRecoveryThreadName(dirPath: String, threadNum: Int, prefix: String = "log-recovery"): String = s"$prefix-$dirPath-$threadNum"
/*
* decrement the number of remaining logs
* @return the number of remaining logs after decremented 1
*/
private[log] def decNumRemainingLogs(numRemainingLogs: ConcurrentMap[String, Int], path: String): Int = {
require(path != null, "path cannot be null to update remaining logs metric.")
numRemainingLogs.compute(path, (_, oldVal) => oldVal - 1)
}
/**
* Recover and load all logs in the given data directories
*/
private[log] def loadLogs(defaultConfig: LogConfig, topicConfigOverrides: Map[String, LogConfig], isStray: UnifiedLog => Boolean): Unit = {
info(s"Loading logs from log dirs $liveLogDirs")
val startMs = time.hiResClockMs()
val threadPools = ArrayBuffer.empty[ExecutorService]
val offlineDirs = mutable.Set.empty[(String, IOException)]
val jobs = ArrayBuffer.empty[Seq[Future[_]]]
var numTotalLogs = 0
// log dir path -> number of Remaining logs map for remainingLogsToRecover metric
val numRemainingLogs: ConcurrentMap[String, Int] = new ConcurrentHashMap[String, Int]
// log recovery thread name -> number of remaining segments map for remainingSegmentsToRecover metric
val numRemainingSegments: ConcurrentMap[String, Int] = new ConcurrentHashMap[String, Int]
def handleIOException(logDirAbsolutePath: String, e: IOException): Unit = {
offlineDirs.add((logDirAbsolutePath, e))
error(s"Error while loading log dir $logDirAbsolutePath", e)
}
val uncleanLogDirs = mutable.Buffer.empty[String]
for (dir <- liveLogDirs) {
val logDirAbsolutePath = dir.getAbsolutePath
var hadCleanShutdown: Boolean = false
try {
val pool = Executors.newFixedThreadPool(numRecoveryThreadsPerDataDir,
new LogRecoveryThreadFactory(logDirAbsolutePath))
threadPools.append(pool)
val cleanShutdownFileHandler = new CleanShutdownFileHandler(dir.getPath)
if (cleanShutdownFileHandler.exists()) {
// Cache the clean shutdown status and use that for rest of log loading workflow. Delete the CleanShutdownFile
// so that if broker crashes while loading the log, it is considered hard shutdown during the next boot up. KAFKA-10471
cleanShutdownFileHandler.delete()
hadCleanShutdown = true
}
hadCleanShutdownFlags.put(logDirAbsolutePath, hadCleanShutdown)
var recoveryPoints = Map[TopicPartition, Long]()
try {
recoveryPoints = this.recoveryPointCheckpoints(dir).read()
} catch {
case e: Exception =>
warn(s"Error occurred while reading recovery-point-offset-checkpoint file of directory " +
s"$logDirAbsolutePath, resetting the recovery checkpoint to 0", e)
}
var logStartOffsets = Map[TopicPartition, Long]()
try {
logStartOffsets = this.logStartOffsetCheckpoints(dir).read()
} catch {
case e: Exception =>
warn(s"Error occurred while reading log-start-offset-checkpoint file of directory " +
s"$logDirAbsolutePath, resetting to the base offset of the first segment", e)
}
val logsToLoad = Option(dir.listFiles).getOrElse(Array.empty).filter(logDir =>
logDir.isDirectory &&
// Ignore remote-log-index-cache directory as that is index cache maintained by tiered storage subsystem
// but not any topic-partition dir.
!logDir.getName.equals(RemoteIndexCache.DIR_NAME) &&
UnifiedLog.parseTopicPartitionName(logDir).topic != KafkaRaftServer.MetadataTopic)
numTotalLogs += logsToLoad.length
numRemainingLogs.put(logDirAbsolutePath, logsToLoad.length)
loadLogsCompletedFlags.put(logDirAbsolutePath, logsToLoad.isEmpty)
if (logsToLoad.isEmpty) {
info(s"No logs found to be loaded in $logDirAbsolutePath")
} else if (hadCleanShutdown) {
info(s"Skipping recovery of ${logsToLoad.length} logs from $logDirAbsolutePath since " +
"clean shutdown file was found")
} else {
info(s"Recovering ${logsToLoad.length} logs from $logDirAbsolutePath since no " +
"clean shutdown file was found")
uncleanLogDirs.append(logDirAbsolutePath)
}
val jobsForDir = logsToLoad.map { logDir =>
val runnable: Runnable = () => {
debug(s"Loading log $logDir")
var log = None: Option[UnifiedLog]
val logLoadStartMs = time.hiResClockMs()
try {
log = Some(loadLog(logDir, hadCleanShutdown, recoveryPoints, logStartOffsets,
defaultConfig, topicConfigOverrides, numRemainingSegments, isStray))
} catch {
case e: IOException =>
handleIOException(logDirAbsolutePath, e)
case e: KafkaStorageException if e.getCause.isInstanceOf[IOException] =>
// KafkaStorageException might be thrown, ex: during writing LeaderEpochFileCache
// And while converting IOException to KafkaStorageException, we've already handled the exception. So we can ignore it here.
} finally {
val logLoadDurationMs = time.hiResClockMs() - logLoadStartMs
val remainingLogs = decNumRemainingLogs(numRemainingLogs, logDirAbsolutePath)
val currentNumLoaded = logsToLoad.length - remainingLogs
log match {
case Some(loadedLog) => info(s"Completed load of $loadedLog with ${loadedLog.numberOfSegments} segments, " +
s"local-log-start-offset ${loadedLog.localLogStartOffset()} and log-end-offset ${loadedLog.logEndOffset} in ${logLoadDurationMs}ms " +
s"($currentNumLoaded/${logsToLoad.length} completed in $logDirAbsolutePath)")
case None => info(s"Error while loading logs in $logDir in ${logLoadDurationMs}ms ($currentNumLoaded/${logsToLoad.length} completed in $logDirAbsolutePath)")
}
if (remainingLogs == 0) {
// loadLog is completed for all logs under the logDdir, mark it.
loadLogsCompletedFlags.put(logDirAbsolutePath, true)
}
}
}
runnable
}
jobs += jobsForDir.map(pool.submit)
} catch {
case e: IOException =>
handleIOException(logDirAbsolutePath, e)
}
}
try {
addLogRecoveryMetrics(numRemainingLogs, numRemainingSegments)
for (dirJobs <- jobs) {
dirJobs.foreach(_.get)
}
offlineDirs.foreach { case (dir, e) =>
logDirFailureChannel.maybeAddOfflineLogDir(dir, s"Error while loading log dir $dir", e)
}
} catch {
case e: ExecutionException =>
error(s"There was an error in one of the threads during logs loading: ${e.getCause}")
throw e.getCause
} finally {
removeLogRecoveryMetrics()
threadPools.foreach(_.shutdown())
}
val elapsedMs = time.hiResClockMs() - startMs
val printedUncleanLogDirs = if (uncleanLogDirs.isEmpty) "" else s" (unclean log dirs = $uncleanLogDirs)"
info(s"Loaded $numTotalLogs logs in ${elapsedMs}ms$printedUncleanLogDirs")
}
private[log] def addLogRecoveryMetrics(numRemainingLogs: ConcurrentMap[String, Int],
numRemainingSegments: ConcurrentMap[String, Int]): Unit = {
debug("Adding log recovery metrics")
for (dir <- logDirs) {
metricsGroup.newGauge("remainingLogsToRecover", () => numRemainingLogs.get(dir.getAbsolutePath),
Map("dir" -> dir.getAbsolutePath).asJava)
for (i <- 0 until numRecoveryThreadsPerDataDir) {
val threadName = logRecoveryThreadName(dir.getAbsolutePath, i)
metricsGroup.newGauge("remainingSegmentsToRecover", () => numRemainingSegments.get(threadName),
Map("dir" -> dir.getAbsolutePath, "threadNum" -> i.toString).asJava)
}
}
}
private[log] def removeLogRecoveryMetrics(): Unit = {
debug("Removing log recovery metrics")
for (dir <- logDirs) {
metricsGroup.removeMetric("remainingLogsToRecover", Map("dir" -> dir.getAbsolutePath).asJava)
for (i <- 0 until numRecoveryThreadsPerDataDir) {
metricsGroup.removeMetric("remainingSegmentsToRecover", Map("dir" -> dir.getAbsolutePath, "threadNum" -> i.toString).asJava)
}
}
}
/**
* Start the background threads to flush logs and do log cleanup
*/
def startup(topicNames: Set[String], isStray: UnifiedLog => Boolean = _ => false): Unit = {
// ensure consistency between default config and overrides
val defaultConfig = currentDefaultConfig
startupWithConfigOverrides(defaultConfig, fetchTopicConfigOverrides(defaultConfig, topicNames), isStray)
}
// visible for testing
@nowarn("cat=deprecation")
private[log] def fetchTopicConfigOverrides(defaultConfig: LogConfig, topicNames: Set[String]): Map[String, LogConfig] = {
val topicConfigOverrides = mutable.Map[String, LogConfig]()
val defaultProps = defaultConfig.originals()
topicNames.foreach { topicName =>
var overrides = configRepository.topicConfig(topicName)
// save memory by only including configs for topics with overrides
if (!overrides.isEmpty) {
Option(overrides.getProperty(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)).foreach { versionString =>
val messageFormatVersion = new MessageFormatVersion(versionString, interBrokerProtocolVersion.version)
if (messageFormatVersion.shouldIgnore) {
val copy = new Properties()
copy.putAll(overrides)
copy.remove(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)
overrides = copy
if (messageFormatVersion.shouldWarn)
warn(messageFormatVersion.topicWarningMessage(topicName))
}
}
val logConfig = LogConfig.fromProps(defaultProps, overrides)
topicConfigOverrides(topicName) = logConfig
}
}
topicConfigOverrides
}
private def fetchLogConfig(topicName: String): LogConfig = {
// ensure consistency between default config and overrides
val defaultConfig = currentDefaultConfig
fetchTopicConfigOverrides(defaultConfig, Set(topicName)).values.headOption.getOrElse(defaultConfig)
}
// visible for testing
private[log] def startupWithConfigOverrides(
defaultConfig: LogConfig,
topicConfigOverrides: Map[String, LogConfig],
isStray: UnifiedLog => Boolean): Unit = {
loadLogs(defaultConfig, topicConfigOverrides, isStray) // this could take a while if shutdown was not clean
/* Schedule the cleanup task to delete old logs */
if (scheduler != null) {
info("Starting log cleanup with a period of %d ms.".format(retentionCheckMs))
scheduler.schedule("kafka-log-retention",
() => cleanupLogs(),
initialTaskDelayMs,
retentionCheckMs)
info("Starting log flusher with a default period of %d ms.".format(flushCheckMs))
scheduler.schedule("kafka-log-flusher",
() => flushDirtyLogs(),
initialTaskDelayMs,
flushCheckMs)
scheduler.schedule("kafka-recovery-point-checkpoint",
() => checkpointLogRecoveryOffsets(),
initialTaskDelayMs,
flushRecoveryOffsetCheckpointMs)
scheduler.schedule("kafka-log-start-offset-checkpoint",
() => checkpointLogStartOffsets(),
initialTaskDelayMs,
flushStartOffsetCheckpointMs)
scheduler.scheduleOnce("kafka-delete-logs", // will be rescheduled after each delete logs with a dynamic period
() => deleteLogs(),
initialTaskDelayMs)
}
if (cleanerConfig.enableCleaner) {
_cleaner = new LogCleaner(cleanerConfig, liveLogDirs, currentLogs, logDirFailureChannel, time = time)
_cleaner.startup()
}
}
/**
* Close all the logs
*/
def shutdown(brokerEpoch: Long = -1): Unit = {
info("Shutting down.")
metricsGroup.removeMetric("OfflineLogDirectoryCount")
for (dir <- logDirs) {
metricsGroup.removeMetric("LogDirectoryOffline", Map("logDirectory" -> dir.getAbsolutePath).asJava)
}
val threadPools = ArrayBuffer.empty[ExecutorService]
val jobs = mutable.Map.empty[File, Seq[Future[_]]]
// stop the cleaner first
if (cleaner != null) {
CoreUtils.swallow(cleaner.shutdown(), this)
}
val localLogsByDir = logsByDir
// close logs in each dir
for (dir <- liveLogDirs) {
debug(s"Flushing and closing logs at $dir")
val pool = Executors.newFixedThreadPool(numRecoveryThreadsPerDataDir,
KafkaThread.nonDaemon(s"log-closing-${dir.getAbsolutePath}", _))
threadPools.append(pool)
val logs = logsInDir(localLogsByDir, dir).values
val jobsForDir = logs.map { log =>
val runnable: Runnable = () => {
// flush the log to ensure latest possible recovery point
log.flush(true)
log.close()
}
runnable
}
jobs(dir) = jobsForDir.map(pool.submit).toSeq
}
try {
jobs.forKeyValue { (dir, dirJobs) =>
if (waitForAllToComplete(dirJobs,
e => warn(s"There was an error in one of the threads during LogManager shutdown: ${e.getCause}"))) {
val logs = logsInDir(localLogsByDir, dir)
// update the last flush point
debug(s"Updating recovery points at $dir")
checkpointRecoveryOffsetsInDir(dir, logs)
debug(s"Updating log start offsets at $dir")
checkpointLogStartOffsetsInDir(dir, logs)
// mark that the shutdown was clean by creating marker file for log dirs that:
// 1. had clean shutdown marker file; or
// 2. had no clean shutdown marker file, but all logs under it have been recovered at startup time
val logDirAbsolutePath = dir.getAbsolutePath
if (hadCleanShutdownFlags.getOrDefault(logDirAbsolutePath, false) ||
loadLogsCompletedFlags.getOrDefault(logDirAbsolutePath, false)) {
val cleanShutdownFileHandler = new CleanShutdownFileHandler(dir.getPath)
debug(s"Writing clean shutdown marker at $dir with broker epoch=$brokerEpoch")
CoreUtils.swallow(cleanShutdownFileHandler.write(brokerEpoch), this)
}
}
}
} finally {
threadPools.foreach(_.shutdown())
// regardless of whether the close succeeded, we need to unlock the data directories
dirLocks.foreach(_.destroy())
}
info("Shutdown complete.")
}
/**
* Truncate the partition logs to the specified offsets and checkpoint the recovery point to this offset
*
* @param partitionOffsets Partition logs that need to be truncated
* @param isFuture True iff the truncation should be performed on the future log of the specified partitions
*/
def truncateTo(partitionOffsets: Map[TopicPartition, Long], isFuture: Boolean): Unit = {
val affectedLogs = ArrayBuffer.empty[UnifiedLog]
for ((topicPartition, truncateOffset) <- partitionOffsets) {
val log = {
if (isFuture)
futureLogs.get(topicPartition)
else
currentLogs.get(topicPartition)
}
// If the log does not exist, skip it
if (log != null) {
// May need to abort and pause the cleaning of the log, and resume after truncation is done.
val needToStopCleaner = truncateOffset < log.activeSegment.baseOffset
if (needToStopCleaner && !isFuture)
abortAndPauseCleaning(topicPartition)
try {
if (log.truncateTo(truncateOffset))
affectedLogs += log
if (needToStopCleaner && !isFuture)
maybeTruncateCleanerCheckpointToActiveSegmentBaseOffset(log, topicPartition)
} finally {
if (needToStopCleaner && !isFuture)
resumeCleaning(topicPartition)
}
}
}
for (dir <- affectedLogs.map(_.parentDirFile).distinct) {
checkpointRecoveryOffsetsInDir(dir)
}
}
/**
* Delete all data in a partition and start the log at the new offset
*
* @param topicPartition The partition whose log needs to be truncated
* @param newOffset The new offset to start the log with
* @param isFuture True iff the truncation should be performed on the future log of the specified partition
* @param logStartOffsetOpt The log start offset to set for the log. If None, the new offset will be used.
*/
def truncateFullyAndStartAt(topicPartition: TopicPartition,
newOffset: Long,
isFuture: Boolean,
logStartOffsetOpt: Option[Long] = None): Unit = {
val log = {
if (isFuture)
futureLogs.get(topicPartition)
else
currentLogs.get(topicPartition)
}
// If the log does not exist, skip it
if (log != null) {
// Abort and pause the cleaning of the log, and resume after truncation is done.
if (!isFuture)
abortAndPauseCleaning(topicPartition)
try {
log.truncateFullyAndStartAt(newOffset, logStartOffsetOpt)
if (!isFuture)
maybeTruncateCleanerCheckpointToActiveSegmentBaseOffset(log, topicPartition)
} finally {
if (!isFuture)
resumeCleaning(topicPartition)
}
checkpointRecoveryOffsetsInDir(log.parentDirFile)
}
}
/**
* Write out the current recovery point for all logs to a text file in the log directory
* to avoid recovering the whole log on startup.
*/
def checkpointLogRecoveryOffsets(): Unit = {
val logsByDirCached = logsByDir
liveLogDirs.foreach { logDir =>
val logsToCheckpoint = logsInDir(logsByDirCached, logDir)
checkpointRecoveryOffsetsInDir(logDir, logsToCheckpoint)
}
}
/**
* Write out the current log start offset for all logs to a text file in the log directory
* to avoid exposing data that have been deleted by DeleteRecordsRequest
*/
def checkpointLogStartOffsets(): Unit = {
val logsByDirCached = logsByDir
liveLogDirs.foreach { logDir =>
checkpointLogStartOffsetsInDir(logDir, logsInDir(logsByDirCached, logDir))
}
}
/**
* Checkpoint recovery offsets for all the logs in logDir.
*
* @param logDir the directory in which the logs to be checkpointed are
*/
// Only for testing
private[log] def checkpointRecoveryOffsetsInDir(logDir: File): Unit = {
checkpointRecoveryOffsetsInDir(logDir, logsInDir(logDir))
}
/**
* Checkpoint recovery offsets for all the provided logs.
*
* @param logDir the directory in which the logs are
* @param logsToCheckpoint the logs to be checkpointed
*/
private def checkpointRecoveryOffsetsInDir(logDir: File, logsToCheckpoint: Map[TopicPartition, UnifiedLog]): Unit = {
try {
recoveryPointCheckpoints.get(logDir).foreach { checkpoint =>
val recoveryOffsets = logsToCheckpoint.map { case (tp, log) => tp -> log.recoveryPoint }
// checkpoint.write calls Utils.atomicMoveWithFallback, which flushes the parent
// directory and guarantees crash consistency.
checkpoint.write(recoveryOffsets)
}
} catch {
case e: KafkaStorageException =>
error(s"Disk error while writing recovery offsets checkpoint in directory $logDir: ${e.getMessage}")
case e: IOException =>
logDirFailureChannel.maybeAddOfflineLogDir(logDir.getAbsolutePath,
s"Disk error while writing recovery offsets checkpoint in directory $logDir: ${e.getMessage}", e)
}
}
/**
* Checkpoint log start offsets for all the provided logs in the provided directory.
*
* @param logDir the directory in which logs are checkpointed
* @param logsToCheckpoint the logs to be checkpointed
*/
private def checkpointLogStartOffsetsInDir(logDir: File, logsToCheckpoint: Map[TopicPartition, UnifiedLog]): Unit = {
try {
logStartOffsetCheckpoints.get(logDir).foreach { checkpoint =>
val logStartOffsets = logsToCheckpoint.collect {
case (tp, log) if log.remoteLogEnabled() || log.logStartOffset > log.logSegments.asScala.head.baseOffset =>
tp -> log.logStartOffset
}
checkpoint.write(logStartOffsets)
}
} catch {
case e: KafkaStorageException =>
error(s"Disk error while writing log start offsets checkpoint in directory $logDir: ${e.getMessage}")
}
}
// The logDir should be an absolute path
def maybeUpdatePreferredLogDir(topicPartition: TopicPartition, logDir: String): Unit = {
// Do not cache the preferred log directory if either the current log or the future log for this partition exists in the specified logDir
if (!getLog(topicPartition).exists(_.parentDir == logDir) &&
!getLog(topicPartition, isFuture = true).exists(_.parentDir == logDir))
preferredLogDirs.put(topicPartition, logDir)
}
/**
* Abort and pause cleaning of the provided partition and log a message about it.
*/
def abortAndPauseCleaning(topicPartition: TopicPartition): Unit = {
if (cleaner != null) {
cleaner.abortAndPauseCleaning(topicPartition)
info(s"The cleaning for partition $topicPartition is aborted and paused")
}
}
/**
* Abort cleaning of the provided partition and log a message about it.
*/
def abortCleaning(topicPartition: TopicPartition): Unit = {
if (cleaner != null) {
cleaner.abortCleaning(topicPartition)
info(s"The cleaning for partition $topicPartition is aborted")
}
}
/**
* Resume cleaning of the provided partition and log a message about it.
*/
private def resumeCleaning(topicPartition: TopicPartition): Unit = {
if (cleaner != null) {
cleaner.resumeCleaning(Seq(topicPartition))
info(s"Cleaning for partition $topicPartition is resumed")
}
}
/**
* Truncate the cleaner's checkpoint to the based offset of the active segment of
* the provided log.
*/
private def maybeTruncateCleanerCheckpointToActiveSegmentBaseOffset(log: UnifiedLog, topicPartition: TopicPartition): Unit = {
if (cleaner != null) {
cleaner.maybeTruncateCheckpoint(log.parentDirFile, topicPartition, log.activeSegment.baseOffset)
}
}
/**
* Get the log if it exists, otherwise return None
*
* @param topicPartition the partition of the log
* @param isFuture True iff the future log of the specified partition should be returned
*/
def getLog(topicPartition: TopicPartition, isFuture: Boolean = false): Option[UnifiedLog] = {
if (isFuture)
Option(futureLogs.get(topicPartition))
else
Option(currentLogs.get(topicPartition))
}
/**
* Method to indicate that logs are getting initialized for the partition passed in as argument.
* This method should always be followed by [[kafka.log.LogManager#finishedInitializingLog]] to indicate that log
* initialization is done.
*/
def initializingLog(topicPartition: TopicPartition): Unit = {
partitionsInitializing(topicPartition) = false
}
/**
* Mark the partition configuration for all partitions that are getting initialized for topic
* as dirty. That will result in reloading of configuration once initialization is done.
*/
def topicConfigUpdated(topic: String): Unit = {
partitionsInitializing.keys.filter(_.topic() == topic).foreach {
topicPartition => partitionsInitializing.replace(topicPartition, false, true)
}
}
/**
* Update the configuration of the provided topic.
*/
def updateTopicConfig(topic: String,
newTopicConfig: Properties,
isRemoteLogStorageSystemEnabled: Boolean): Unit = {
topicConfigUpdated(topic)
val logs = logsByTopic(topic)
// Combine the default properties with the overrides in zk to create the new LogConfig
val newLogConfig = LogConfig.fromProps(currentDefaultConfig.originals, newTopicConfig)
// We would like to validate the configuration no matter whether the logs have materialised on disk or not.
// Otherwise we risk someone creating a tiered-topic, disabling Tiered Storage cluster-wide and the check
// failing since the logs for the topic are non-existent.
LogConfig.validateRemoteStorageOnlyIfSystemEnabled(newLogConfig.values(), isRemoteLogStorageSystemEnabled, true)
if (logs.nonEmpty) {
logs.foreach { log =>
val oldLogConfig = log.updateConfig(newLogConfig)
if (oldLogConfig.compact && !newLogConfig.compact) {
abortCleaning(log.topicPartition)
}
}
}
}
/**
* Mark all in progress partitions having dirty configuration if broker configuration is updated.
*/
def brokerConfigUpdated(): Unit = {
partitionsInitializing.keys.foreach {
topicPartition => partitionsInitializing.replace(topicPartition, false, true)
}
}
/**
* Method to indicate that the log initialization for the partition passed in as argument is
* finished. This method should follow a call to [[kafka.log.LogManager#initializingLog]].
*
* It will retrieve the topic configs a second time if they were updated while the
* relevant log was being loaded.