Skip to content

Commit c92e62a

Browse files
authored
MINOR: Add log identifier/prefix printing in Log layer static functions (#10742)
When #10478 was merged, we accidentally lost the identifier/prefix string that we used to previously log to stderr from some of the functions in the Log class. In this PR, I have reinstated the identifier/prefix logging in these functions, so that the debuggability is restored. Reviewers: Luke Chen <[email protected]>, Cong Ding <[email protected]>, Jun Rao <[email protected]>
1 parent efb7cda commit c92e62a

File tree

9 files changed

+71
-51
lines changed

9 files changed

+71
-51
lines changed

core/src/main/scala/kafka/log/Log.scala

+36-21
Original file line numberDiff line numberDiff line change
@@ -557,7 +557,7 @@ class Log(@volatile private var _dir: File,
557557
}
558558

559559
private def initializeLeaderEpochCache(): Unit = lock synchronized {
560-
leaderEpochCache = Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, recordVersion)
560+
leaderEpochCache = Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, recordVersion, logIdent)
561561
}
562562

563563
private def updateLogEndOffset(offset: Long): Unit = {
@@ -592,7 +592,7 @@ class Log(@volatile private var _dir: File,
592592
producerStateManager: ProducerStateManager): Unit = lock synchronized {
593593
checkIfMemoryMappedBufferClosed()
594594
Log.rebuildProducerState(producerStateManager, segments, logStartOffset, lastOffset, recordVersion, time,
595-
reloadFromCleanShutdown = false)
595+
reloadFromCleanShutdown = false, logIdent)
596596
}
597597

598598
def activeProducers: Seq[DescribeProducersResponseData.ProducerState] = {
@@ -1888,14 +1888,14 @@ class Log(@volatile private var _dir: File,
18881888

18891889
private def deleteSegmentFiles(segments: Iterable[LogSegment], asyncDelete: Boolean, deleteProducerStateSnapshots: Boolean = true): Unit = {
18901890
Log.deleteSegmentFiles(segments, asyncDelete, deleteProducerStateSnapshots, dir, topicPartition,
1891-
config, scheduler, logDirFailureChannel, producerStateManager)
1891+
config, scheduler, logDirFailureChannel, producerStateManager, this.logIdent)
18921892
}
18931893

18941894
private[log] def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Unit = {
18951895
lock synchronized {
18961896
checkIfMemoryMappedBufferClosed()
18971897
Log.replaceSegments(segments, newSegments, oldSegments, isRecoveredSwapFile, dir, topicPartition,
1898-
config, scheduler, logDirFailureChannel, producerStateManager)
1898+
config, scheduler, logDirFailureChannel, producerStateManager, this.logIdent)
18991899
}
19001900
}
19011901

@@ -1937,7 +1937,7 @@ class Log(@volatile private var _dir: File,
19371937
}
19381938

19391939
private[log] def splitOverflowedSegment(segment: LogSegment): List[LogSegment] = lock synchronized {
1940-
Log.splitOverflowedSegment(segment, segments, dir, topicPartition, config, scheduler, logDirFailureChannel, producerStateManager)
1940+
Log.splitOverflowedSegment(segment, segments, dir, topicPartition, config, scheduler, logDirFailureChannel, producerStateManager, this.logIdent)
19411941
}
19421942

19431943
}
@@ -2005,7 +2005,12 @@ object Log extends Logging {
20052005
Files.createDirectories(dir.toPath)
20062006
val topicPartition = Log.parseTopicPartitionName(dir)
20072007
val segments = new LogSegments(topicPartition)
2008-
val leaderEpochCache = Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion)
2008+
val leaderEpochCache = Log.maybeCreateLeaderEpochCache(
2009+
dir,
2010+
topicPartition,
2011+
logDirFailureChannel,
2012+
config.messageFormatVersion.recordVersion,
2013+
s"[Log partition=$topicPartition, dir=${dir.getParent}] ")
20092014
val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
20102015
val offsets = LogLoader.load(LoadLogParams(
20112016
dir,
@@ -2226,12 +2231,14 @@ object Log extends Logging {
22262231
* @param topicPartition The topic partition
22272232
* @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure
22282233
* @param recordVersion The record version
2234+
* @param logPrefix The logging prefix
22292235
* @return The new LeaderEpochFileCache instance (if created), none otherwise
22302236
*/
22312237
def maybeCreateLeaderEpochCache(dir: File,
22322238
topicPartition: TopicPartition,
22332239
logDirFailureChannel: LogDirFailureChannel,
2234-
recordVersion: RecordVersion): Option[LeaderEpochFileCache] = {
2240+
recordVersion: RecordVersion,
2241+
logPrefix: String): Option[LeaderEpochFileCache] = {
22352242
val leaderEpochFile = LeaderEpochCheckpointFile.newFile(dir)
22362243

22372244
def newLeaderEpochFileCache(): LeaderEpochFileCache = {
@@ -2246,7 +2253,7 @@ object Log extends Logging {
22462253
None
22472254

22482255
if (currentCache.exists(_.nonEmpty))
2249-
warn(s"Deleting non-empty leader epoch cache due to incompatible message format $recordVersion")
2256+
warn(s"${logPrefix}Deleting non-empty leader epoch cache due to incompatible message format $recordVersion")
22502257

22512258
Files.deleteIfExists(leaderEpochFile.toPath)
22522259
None
@@ -2293,6 +2300,7 @@ object Log extends Logging {
22932300
* @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure
22942301
* @param producerStateManager The ProducerStateManager instance (if any) containing state associated
22952302
* with the existingSegments
2303+
* @param logPrefix The logging prefix
22962304
*/
22972305
private[log] def replaceSegments(existingSegments: LogSegments,
22982306
newSegments: Seq[LogSegment],
@@ -2303,7 +2311,8 @@ object Log extends Logging {
23032311
config: LogConfig,
23042312
scheduler: Scheduler,
23052313
logDirFailureChannel: LogDirFailureChannel,
2306-
producerStateManager: ProducerStateManager): Unit = {
2314+
producerStateManager: ProducerStateManager,
2315+
logPrefix: String): Unit = {
23072316
val sortedNewSegments = newSegments.sortBy(_.baseOffset)
23082317
// Some old segments may have been removed from index and scheduled for async deletion after the caller reads segments
23092318
// but before this method is executed. We want to filter out those segments to avoid calling asyncDeleteSegment()
@@ -2332,7 +2341,8 @@ object Log extends Logging {
23322341
config,
23332342
scheduler,
23342343
logDirFailureChannel,
2335-
producerStateManager)
2344+
producerStateManager,
2345+
logPrefix)
23362346
}
23372347
// okay we are safe now, remove the swap suffix
23382348
sortedNewSegments.foreach(_.changeFileSuffixes(Log.SwapFileSuffix, ""))
@@ -2359,7 +2369,7 @@ object Log extends Logging {
23592369
* @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure
23602370
* @param producerStateManager The ProducerStateManager instance (if any) containing state associated
23612371
* with the existingSegments
2362-
*
2372+
* @param logPrefix The logging prefix
23632373
* @throws IOException if the file can't be renamed and still exists
23642374
*/
23652375
private[log] def deleteSegmentFiles(segmentsToDelete: Iterable[LogSegment],
@@ -2370,11 +2380,12 @@ object Log extends Logging {
23702380
config: LogConfig,
23712381
scheduler: Scheduler,
23722382
logDirFailureChannel: LogDirFailureChannel,
2373-
producerStateManager: ProducerStateManager): Unit = {
2383+
producerStateManager: ProducerStateManager,
2384+
logPrefix: String): Unit = {
23742385
segmentsToDelete.foreach(_.changeFileSuffixes("", Log.DeletedFileSuffix))
23752386

23762387
def deleteSegments(): Unit = {
2377-
info(s"Deleting segment files ${segmentsToDelete.mkString(",")}")
2388+
info(s"${logPrefix}Deleting segment files ${segmentsToDelete.mkString(",")}")
23782389
val parentDir = dir.getParent
23792390
maybeHandleIOException(logDirFailureChannel, parentDir, s"Error while deleting segments for $topicPartition in dir $parentDir") {
23802391
segmentsToDelete.foreach { segment =>
@@ -2429,14 +2440,16 @@ object Log extends Logging {
24292440
* @param time The time instance used for checking the clock
24302441
* @param reloadFromCleanShutdown True if the producer state is being built after a clean shutdown,
24312442
* false otherwise.
2443+
* @param logPrefix The logging prefix
24322444
*/
24332445
private[log] def rebuildProducerState(producerStateManager: ProducerStateManager,
24342446
segments: LogSegments,
24352447
logStartOffset: Long,
24362448
lastOffset: Long,
24372449
recordVersion: RecordVersion,
24382450
time: Time,
2439-
reloadFromCleanShutdown: Boolean): Unit = {
2451+
reloadFromCleanShutdown: Boolean,
2452+
logPrefix: String): Unit = {
24402453
val allSegments = segments.values
24412454
val offsetsToSnapshot =
24422455
if (allSegments.nonEmpty) {
@@ -2445,7 +2458,7 @@ object Log extends Logging {
24452458
} else {
24462459
Seq(Some(lastOffset))
24472460
}
2448-
info(s"Loading producer state till offset $lastOffset with message format version ${recordVersion.value}")
2461+
info(s"${logPrefix}Loading producer state till offset $lastOffset with message format version ${recordVersion.value}")
24492462

24502463
// We want to avoid unnecessary scanning of the log to build the producer state when the broker is being
24512464
// upgraded. The basic idea is to use the absence of producer snapshot files to detect the upgrade case,
@@ -2469,7 +2482,7 @@ object Log extends Logging {
24692482
producerStateManager.takeSnapshot()
24702483
}
24712484
} else {
2472-
info(s"Reloading from producer snapshot and rebuilding producer state from offset $lastOffset")
2485+
info(s"${logPrefix}Reloading from producer snapshot and rebuilding producer state from offset $lastOffset")
24732486
val isEmptyBeforeTruncation = producerStateManager.isEmpty && producerStateManager.mapEndOffset >= lastOffset
24742487
val producerStateLoadStart = time.milliseconds()
24752488
producerStateManager.truncateAndReload(logStartOffset, lastOffset, time.milliseconds())
@@ -2508,7 +2521,7 @@ object Log extends Logging {
25082521
}
25092522
producerStateManager.updateMapEndOffset(lastOffset)
25102523
producerStateManager.takeSnapshot()
2511-
info(s"Producer state recovery took ${segmentRecoveryStart - producerStateLoadStart}ms for snapshot load " +
2524+
info(s"${logPrefix}Producer state recovery took ${segmentRecoveryStart - producerStateLoadStart}ms for snapshot load " +
25122525
s"and ${time.milliseconds() - segmentRecoveryStart}ms for segment recovery from offset $lastOffset")
25132526
}
25142527
}
@@ -2535,6 +2548,7 @@ object Log extends Logging {
25352548
* @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure
25362549
* @param producerStateManager The ProducerStateManager instance (if any) containing state associated
25372550
* with the existingSegments
2551+
* @param logPrefix The logging prefix
25382552
* @return List of new segments that replace the input segment
25392553
*/
25402554
private[log] def splitOverflowedSegment(segment: LogSegment,
@@ -2544,11 +2558,12 @@ object Log extends Logging {
25442558
config: LogConfig,
25452559
scheduler: Scheduler,
25462560
logDirFailureChannel: LogDirFailureChannel,
2547-
producerStateManager: ProducerStateManager): List[LogSegment] = {
2561+
producerStateManager: ProducerStateManager,
2562+
logPrefix: String): List[LogSegment] = {
25482563
require(Log.isLogFile(segment.log.file), s"Cannot split file ${segment.log.file.getAbsoluteFile}")
25492564
require(segment.hasOverflow, "Split operation is only permitted for segments with overflow")
25502565

2551-
info(s"Splitting overflowed segment $segment")
2566+
info(s"${logPrefix}Splitting overflowed segment $segment")
25522567

25532568
val newSegments = ListBuffer[LogSegment]()
25542569
try {
@@ -2581,9 +2596,9 @@ object Log extends Logging {
25812596
s" before: ${segment.log.sizeInBytes} after: $totalSizeOfNewSegments")
25822597

25832598
// replace old segment with new ones
2584-
info(s"Replacing overflowed segment $segment with split segments $newSegments")
2599+
info(s"${logPrefix}Replacing overflowed segment $segment with split segments $newSegments")
25852600
replaceSegments(existingSegments, newSegments.toList, List(segment), isRecoveredSwapFile = false,
2586-
dir, topicPartition, config, scheduler, logDirFailureChannel, producerStateManager)
2601+
dir, topicPartition, config, scheduler, logDirFailureChannel, producerStateManager, logPrefix)
25872602
newSegments.toList
25882603
} catch {
25892604
case e: Exception =>

0 commit comments

Comments
 (0)