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 ,
@@ -1062,70 +1063,71 @@ private class StateUpdaterMetrics {
1062
1063
private final Sensor standbyRestoreRatioSensor ;
1063
1064
private final Sensor checkpointRatioSensor ;
1064
1065
1065
- private final Deque <String > allSensorNames = new LinkedList <>();
1066
+ private final Deque <Sensor > allSensors = new LinkedList <>();
1066
1067
private final Deque <MetricName > allMetricNames = new LinkedList <>();
1067
1068
1068
- private StateUpdaterMetrics (final Metrics metrics , final String threadId ) {
1069
+ private StateUpdaterMetrics (final StreamsMetricsImpl metrics , final String threadId ) {
1069
1070
final Map <String , String > threadLevelTags = new LinkedHashMap <>();
1070
1071
threadLevelTags .put (THREAD_ID_TAG , threadId );
1072
+ final Metrics metricsRegistry = metrics .metricsRegistry ();
1071
1073
1072
- MetricName metricName = metrics .metricName ("active-restoring-tasks" ,
1074
+ MetricName metricName = metricsRegistry .metricName ("active-restoring-tasks" ,
1073
1075
STATE_LEVEL_GROUP ,
1074
1076
"The number of active tasks currently undergoing restoration" ,
1075
1077
threadLevelTags );
1076
- metrics .addMetric (metricName , (config , now ) -> stateUpdaterThread != null ?
1078
+ metricsRegistry .addMetric (metricName , (config , now ) -> stateUpdaterThread != null ?
1077
1079
stateUpdaterThread .numRestoringActiveTasks () : 0 );
1078
1080
allMetricNames .push (metricName );
1079
1081
1080
- metricName = metrics .metricName ("standby-updating-tasks" ,
1082
+ metricName = metricsRegistry .metricName ("standby-updating-tasks" ,
1081
1083
STATE_LEVEL_GROUP ,
1082
1084
"The number of standby tasks currently undergoing state update" ,
1083
1085
threadLevelTags );
1084
- metrics .addMetric (metricName , (config , now ) -> stateUpdaterThread != null ?
1086
+ metricsRegistry .addMetric (metricName , (config , now ) -> stateUpdaterThread != null ?
1085
1087
stateUpdaterThread .numUpdatingStandbyTasks () : 0 );
1086
1088
allMetricNames .push (metricName );
1087
1089
1088
- metricName = metrics .metricName ("active-paused-tasks" ,
1090
+ metricName = metricsRegistry .metricName ("active-paused-tasks" ,
1089
1091
STATE_LEVEL_GROUP ,
1090
1092
"The number of active tasks paused restoring" ,
1091
1093
threadLevelTags );
1092
- metrics .addMetric (metricName , (config , now ) -> stateUpdaterThread != null ?
1094
+ metricsRegistry .addMetric (metricName , (config , now ) -> stateUpdaterThread != null ?
1093
1095
stateUpdaterThread .numPausedActiveTasks () : 0 );
1094
1096
allMetricNames .push (metricName );
1095
1097
1096
- metricName = metrics .metricName ("standby-paused-tasks" ,
1098
+ metricName = metricsRegistry .metricName ("standby-paused-tasks" ,
1097
1099
STATE_LEVEL_GROUP ,
1098
1100
"The number of standby tasks paused state update" ,
1099
1101
threadLevelTags );
1100
- metrics .addMetric (metricName , (config , now ) -> stateUpdaterThread != null ?
1102
+ metricsRegistry .addMetric (metricName , (config , now ) -> stateUpdaterThread != null ?
1101
1103
stateUpdaterThread .numPausedStandbyTasks () : 0 );
1102
1104
allMetricNames .push (metricName );
1103
1105
1104
- this .idleRatioSensor = metrics .sensor ( "idle-ratio" , RecordingLevel .INFO );
1106
+ this .idleRatioSensor = metrics .threadLevelSensor ( threadId , "idle-ratio" , RecordingLevel .INFO );
1105
1107
this .idleRatioSensor .add (new MetricName ("idle-ratio" , STATE_LEVEL_GROUP , IDLE_RATIO_DESCRIPTION , threadLevelTags ), new Avg ());
1106
- allSensorNames .add ("idle-ratio" );
1108
+ allSensors .add (this . idleRatioSensor );
1107
1109
1108
- this .activeRestoreRatioSensor = metrics .sensor ( "active-restore-ratio" , RecordingLevel .INFO );
1110
+ this .activeRestoreRatioSensor = metrics .threadLevelSensor ( threadId , "active-restore-ratio" , RecordingLevel .INFO );
1109
1111
this .activeRestoreRatioSensor .add (new MetricName ("active-restore-ratio" , STATE_LEVEL_GROUP , RESTORE_RATIO_DESCRIPTION , threadLevelTags ), new Avg ());
1110
- allSensorNames .add ("active-restore-ratio" );
1112
+ allSensors .add (this . activeRestoreRatioSensor );
1111
1113
1112
- this .standbyRestoreRatioSensor = metrics .sensor ( "standby-update-ratio" , RecordingLevel .INFO );
1114
+ this .standbyRestoreRatioSensor = metrics .threadLevelSensor ( threadId , "standby-update-ratio" , RecordingLevel .INFO );
1113
1115
this .standbyRestoreRatioSensor .add (new MetricName ("standby-update-ratio" , STATE_LEVEL_GROUP , UPDATE_RATIO_DESCRIPTION , threadLevelTags ), new Avg ());
1114
- allSensorNames .add ("standby-update-ratio" );
1116
+ allSensors .add (this . standbyRestoreRatioSensor );
1115
1117
1116
- this .checkpointRatioSensor = metrics .sensor ( "checkpoint-ratio" , RecordingLevel .INFO );
1118
+ this .checkpointRatioSensor = metrics .threadLevelSensor ( threadId , "checkpoint-ratio" , RecordingLevel .INFO );
1117
1119
this .checkpointRatioSensor .add (new MetricName ("checkpoint-ratio" , STATE_LEVEL_GROUP , CHECKPOINT_RATIO_DESCRIPTION , threadLevelTags ), new Avg ());
1118
- allSensorNames .add ("checkpoint-ratio" );
1120
+ allSensors .add (this . checkpointRatioSensor );
1119
1121
1120
- this .restoreSensor = metrics .sensor ( "restore-records" , RecordingLevel .INFO );
1122
+ this .restoreSensor = metrics .threadLevelSensor ( threadId , "restore-records" , RecordingLevel .INFO );
1121
1123
this .restoreSensor .add (new MetricName ("restore-records-rate" , STATE_LEVEL_GROUP , RESTORE_RECORDS_RATE_DESCRIPTION , threadLevelTags ), new Rate ());
1122
1124
this .restoreSensor .add (new MetricName ("restore-call-rate" , STATE_LEVEL_GROUP , RESTORE_RATE_DESCRIPTION , threadLevelTags ), new Rate (new WindowedCount ()));
1123
- allSensorNames .add ("restore-records" );
1125
+ allSensors .add (this . restoreSensor );
1124
1126
}
1125
1127
1126
1128
void clear () {
1127
- while (!allSensorNames .isEmpty ()) {
1128
- metrics .removeSensor (allSensorNames .pop ());
1129
+ while (!allSensors .isEmpty ()) {
1130
+ metrics .removeSensor (allSensors .pop ());
1129
1131
}
1130
1132
1131
1133
while (!allMetricNames .isEmpty ()) {
0 commit comments