@ -237,7 +237,7 @@ case object SnapshotGenerated extends LogStartOffsetIncrementReason {
@@ -237,7 +237,7 @@ case object SnapshotGenerated extends LogStartOffsetIncrementReason {
*/
@threadsafe
class UnifiedLog ( @volatile var logStartOffset : Long ,
private val localLog : LocalLog ,
private [ log ] val localLog : LocalLog ,
brokerTopicStats : BrokerTopicStats ,
val producerIdExpirationCheckIntervalMs : Int ,
@volatile var leaderEpochCache : Option [ LeaderEpochFileCache ] ,
@ -552,15 +552,23 @@ class UnifiedLog(@volatile var logStartOffset: Long,
@@ -552,15 +552,23 @@ class UnifiedLog(@volatile var logStartOffset: Long,
)
}
private val tags = {
val maybeFutureTag = if ( isFuture ) Map ( "is-future" -> "true" ) else Map . empty [ String , String ]
Map ( "topic" -> topicPartition . topic , "partition" -> topicPartition . partition . toString ) ++ maybeFutureTag
}
newGauge ( LogMetricNames . NumLogSegments , ( ) => numberOfSegments , tags )
newGauge ( LogMetricNames . LogStartOffset , ( ) => logStartOffset , tags )
newGauge ( LogMetricNames . LogEndOffset , ( ) => logEndOffset , tags )
newGauge ( LogMetricNames . Size , ( ) => size , tags )
private var metricNames : Map [ String , Map [ String , String ] ] = Map . empty
newMetrics ( )
private [ log ] def newMetrics ( ) : Unit = {
val tags = Map ( "topic" -> topicPartition . topic , "partition" -> topicPartition . partition . toString ) ++
( if ( isFuture ) Map ( "is-future" -> "true" ) else Map . empty )
newGauge ( LogMetricNames . NumLogSegments , ( ) => numberOfSegments , tags )
newGauge ( LogMetricNames . LogStartOffset , ( ) => logStartOffset , tags )
newGauge ( LogMetricNames . LogEndOffset , ( ) => logEndOffset , tags )
newGauge ( LogMetricNames . Size , ( ) => size , tags )
metricNames = Map ( LogMetricNames . NumLogSegments -> tags ,
LogMetricNames . LogStartOffset -> tags ,
LogMetricNames . LogEndOffset -> tags ,
LogMetricNames . Size -> tags )
}
val producerExpireCheck = scheduler . schedule ( name = "PeriodicProducerExpirationCheck" , fun = ( ) => {
lock synchronized {
@ -1795,10 +1803,10 @@ class UnifiedLog(@volatile var logStartOffset: Long,
@@ -1795,10 +1803,10 @@ class UnifiedLog(@volatile var logStartOffset: Long,
* remove deleted log metrics
*/
private [ log ] def removeLogMetrics ( ) : Unit = {
re moveMetric ( LogM etricNames. NumLogSegments , tags )
removeMetric ( LogMetricNames . LogStartOffset , tags )
removeMetric ( LogMetricNames . LogEndOffset , tags )
removeMetric ( LogMetricNames . Size , tags )
metricNames . foreach {
case ( name , tags ) => removeMetric ( name , tags )
}
metricNames = Map . empty
}
/* *