36
36
import org .apache .kafka .streams .processor .TaskId ;
37
37
import org .apache .kafka .streams .processor .internals .Task .State ;
38
38
import org .apache .kafka .streams .processor .internals .TaskAndAction .Action ;
39
+ import org .apache .kafka .streams .processor .internals .metrics .StreamsMetricsImpl ;
39
40
40
41
import org .slf4j .Logger ;
41
42
@@ -89,7 +90,7 @@ private class StateUpdaterThread extends Thread {
89
90
private volatile KafkaFutureImpl <Uuid > clientInstanceIdFuture = new KafkaFutureImpl <>();
90
91
91
92
public StateUpdaterThread (final String name ,
92
- final Metrics metrics ,
93
+ final StreamsMetricsImpl metrics ,
93
94
final ChangelogReader changelogReader ) {
94
95
super (name );
95
96
this .changelogReader = changelogReader ;
@@ -745,7 +746,7 @@ private void recordMetrics(final long now, final long totalLatency, final long t
745
746
private final Time time ;
746
747
private final Logger log ;
747
748
private final String name ;
748
- private final Metrics metrics ;
749
+ private final StreamsMetricsImpl metrics ;
749
750
private final Consumer <byte [], byte []> restoreConsumer ;
750
751
private final ChangelogReader changelogReader ;
751
752
private final TopologyMetadata topologyMetadata ;
@@ -766,7 +767,7 @@ private void recordMetrics(final long now, final long totalLatency, final long t
766
767
private StateUpdaterThread stateUpdaterThread = null ;
767
768
768
769
public DefaultStateUpdater (final String name ,
769
- final Metrics metrics ,
770
+ final StreamsMetricsImpl metrics ,
770
771
final StreamsConfig config ,
771
772
final Consumer <byte [], byte []> restoreConsumer ,
772
773
final ChangelogReader changelogReader ,
@@ -1059,74 +1060,75 @@ private class StateUpdaterMetrics {
1059
1060
private final Sensor standbyRestoreRatioSensor ;
1060
1061
private final Sensor checkpointRatioSensor ;
1061
1062
1062
- private final Deque <String > allSensorNames = new LinkedList <>();
1063
+ private final Deque <Sensor > allSensors = new LinkedList <>();
1063
1064
private final Deque <MetricName > allMetricNames = new LinkedList <>();
1064
1065
1065
- private StateUpdaterMetrics (final Metrics metrics , final String threadId ) {
1066
+ private StateUpdaterMetrics (final StreamsMetricsImpl metrics , final String threadId ) {
1066
1067
final Map <String , String > threadLevelTags = new LinkedHashMap <>();
1067
1068
threadLevelTags .put (THREAD_ID_TAG , threadId );
1069
+ final Metrics metricsRegistry = metrics .metricsRegistry ();
1068
1070
1069
- MetricName metricName = metrics .metricName ("active-restoring-tasks" ,
1071
+ MetricName metricName = metricsRegistry .metricName ("active-restoring-tasks" ,
1070
1072
STATE_LEVEL_GROUP ,
1071
1073
"The number of active tasks currently undergoing restoration" ,
1072
1074
threadLevelTags );
1073
- metrics .addMetric (metricName , (config , now ) -> stateUpdaterThread != null ?
1075
+ metricsRegistry .addMetric (metricName , (config , now ) -> stateUpdaterThread != null ?
1074
1076
stateUpdaterThread .numRestoringActiveTasks () : 0 );
1075
1077
allMetricNames .push (metricName );
1076
1078
1077
- metricName = metrics .metricName ("standby-updating-tasks" ,
1079
+ metricName = metricsRegistry .metricName ("standby-updating-tasks" ,
1078
1080
STATE_LEVEL_GROUP ,
1079
1081
"The number of standby tasks currently undergoing state update" ,
1080
1082
threadLevelTags );
1081
- metrics .addMetric (metricName , (config , now ) -> stateUpdaterThread != null ?
1083
+ metricsRegistry .addMetric (metricName , (config , now ) -> stateUpdaterThread != null ?
1082
1084
stateUpdaterThread .numUpdatingStandbyTasks () : 0 );
1083
1085
allMetricNames .push (metricName );
1084
1086
1085
- metricName = metrics .metricName ("active-paused-tasks" ,
1087
+ metricName = metricsRegistry .metricName ("active-paused-tasks" ,
1086
1088
STATE_LEVEL_GROUP ,
1087
1089
"The number of active tasks paused restoring" ,
1088
1090
threadLevelTags );
1089
- metrics .addMetric (metricName , (config , now ) -> stateUpdaterThread != null ?
1091
+ metricsRegistry .addMetric (metricName , (config , now ) -> stateUpdaterThread != null ?
1090
1092
stateUpdaterThread .numPausedActiveTasks () : 0 );
1091
1093
allMetricNames .push (metricName );
1092
1094
1093
- metricName = metrics .metricName ("standby-paused-tasks" ,
1095
+ metricName = metricsRegistry .metricName ("standby-paused-tasks" ,
1094
1096
STATE_LEVEL_GROUP ,
1095
1097
"The number of standby tasks paused state update" ,
1096
1098
threadLevelTags );
1097
- metrics .addMetric (metricName , (config , now ) -> stateUpdaterThread != null ?
1099
+ metricsRegistry .addMetric (metricName , (config , now ) -> stateUpdaterThread != null ?
1098
1100
stateUpdaterThread .numPausedStandbyTasks () : 0 );
1099
1101
allMetricNames .push (metricName );
1100
1102
1101
- this .idleRatioSensor = metrics .sensor ( "idle-ratio" , RecordingLevel .INFO );
1103
+ this .idleRatioSensor = metrics .threadLevelSensor ( threadId , "idle-ratio" , RecordingLevel .INFO );
1102
1104
this .idleRatioSensor .add (new MetricName ("idle-ratio" , STATE_LEVEL_GROUP , IDLE_RATIO_DESCRIPTION , threadLevelTags ), new Avg ());
1103
- allSensorNames .add ("idle-ratio" );
1105
+ allSensors .add (this . idleRatioSensor );
1104
1106
1105
- this .activeRestoreRatioSensor = metrics .sensor ( "active-restore-ratio" , RecordingLevel .INFO );
1107
+ this .activeRestoreRatioSensor = metrics .threadLevelSensor ( threadId , "active-restore-ratio" , RecordingLevel .INFO );
1106
1108
this .activeRestoreRatioSensor .add (new MetricName ("active-restore-ratio" , STATE_LEVEL_GROUP , RESTORE_RATIO_DESCRIPTION , threadLevelTags ), new Avg ());
1107
- allSensorNames .add ("active-restore-ratio" );
1109
+ allSensors .add (this . activeRestoreRatioSensor );
1108
1110
1109
- this .standbyRestoreRatioSensor = metrics .sensor ( "standby-update-ratio" , RecordingLevel .INFO );
1111
+ this .standbyRestoreRatioSensor = metrics .threadLevelSensor ( threadId , "standby-update-ratio" , RecordingLevel .INFO );
1110
1112
this .standbyRestoreRatioSensor .add (new MetricName ("standby-update-ratio" , STATE_LEVEL_GROUP , UPDATE_RATIO_DESCRIPTION , threadLevelTags ), new Avg ());
1111
- allSensorNames .add ("standby-update-ratio" );
1113
+ allSensors .add (this . standbyRestoreRatioSensor );
1112
1114
1113
- this .checkpointRatioSensor = metrics .sensor ( "checkpoint-ratio" , RecordingLevel .INFO );
1115
+ this .checkpointRatioSensor = metrics .threadLevelSensor ( threadId , "checkpoint-ratio" , RecordingLevel .INFO );
1114
1116
this .checkpointRatioSensor .add (new MetricName ("checkpoint-ratio" , STATE_LEVEL_GROUP , CHECKPOINT_RATIO_DESCRIPTION , threadLevelTags ), new Avg ());
1115
- allSensorNames .add ("checkpoint-ratio" );
1117
+ allSensors .add (this . checkpointRatioSensor );
1116
1118
1117
- this .restoreSensor = metrics .sensor ( "restore-records" , RecordingLevel .INFO );
1119
+ this .restoreSensor = metrics .threadLevelSensor ( threadId , "restore-records" , RecordingLevel .INFO );
1118
1120
this .restoreSensor .add (new MetricName ("restore-records-rate" , STATE_LEVEL_GROUP , RESTORE_RECORDS_RATE_DESCRIPTION , threadLevelTags ), new Rate ());
1119
1121
this .restoreSensor .add (new MetricName ("restore-call-rate" , STATE_LEVEL_GROUP , RESTORE_RATE_DESCRIPTION , threadLevelTags ), new Rate (new WindowedCount ()));
1120
- allSensorNames .add ("restore-records" );
1122
+ allSensors .add (this . restoreSensor );
1121
1123
}
1122
1124
1123
1125
void clear () {
1124
- while (!allSensorNames .isEmpty ()) {
1125
- metrics .removeSensor (allSensorNames .pop ());
1126
+ while (!allSensors .isEmpty ()) {
1127
+ metrics .removeSensor (allSensors .pop ());
1126
1128
}
1127
1129
1128
1130
while (!allMetricNames .isEmpty ()) {
1129
- metrics .removeMetric (allMetricNames .pop ());
1131
+ metrics .metricsRegistry (). removeMetric (allMetricNames .pop ());
1130
1132
}
1131
1133
}
1132
1134
}
0 commit comments