aliehsaeedii commented on code in PR #21201:
URL: https://github.com/apache/kafka/pull/21201#discussion_r2686361892
##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdater.java:
##########
@@ -713,19 +726,65 @@ private void measureCheckpointLatency(final Runnable
actionToMeasure) {
private void recordMetrics(final long now, final long totalLatency,
final long totalWaitLatency) {
final long totalRestoreLatency = Math.max(0L, totalLatency -
totalWaitLatency - totalCheckpointLatency);
- updaterMetrics.idleRatioSensor.record((double) totalWaitLatency /
totalLatency, now);
- updaterMetrics.checkpointRatioSensor.record((double)
totalCheckpointLatency / totalLatency, now);
+ recordWindowedSum(
+ now,
+ (double) totalWaitLatency,
+ (double) totalCheckpointLatency,
+ (double) totalRestoreLatency *
(changelogReader.isRestoringActive() ? 1.0d : 0.0d),
+ (double) totalRestoreLatency *
(changelogReader.isRestoringActive() ? 0.0d : 1.0d)
+ );
- if (changelogReader.isRestoringActive()) {
- updaterMetrics.activeRestoreRatioSensor.record((double)
totalRestoreLatency / totalLatency, now);
- updaterMetrics.standbyRestoreRatioSensor.record(0.0d, now);
- } else {
- updaterMetrics.standbyRestoreRatioSensor.record((double)
totalRestoreLatency / totalLatency, now);
- updaterMetrics.activeRestoreRatioSensor.record(0.0d, now);
- }
+ recordRatios(now);
totalCheckpointLatency = 0L;
}
+
+ private void initTimeWindowIfNeeded(final long now) {
+ if (!timeWindowInitialized) {
+ idleTimeWindowedSum.record(metricsConfig, 0.0, now);
+ checkpointTimeWindowedSum.record(metricsConfig, 0.0, now);
+ activeRestoreTimeWindowedSum.record(metricsConfig, 0.0, now);
+ standbyRestoreTimeWindowedSum.record(metricsConfig, 0.0, now);
+ timeWindowInitialized = true;
+ }
+ }
+
+ private void recordWindowedSum(final long now,
+ final double idleTime,
+ final double checkpointTime,
+ final double activeRestoreTime,
+ final double standbyRestoreTime) {
+ idleTimeWindowedSum.record(metricsConfig, idleTime, now);
+ checkpointTimeWindowedSum.record(metricsConfig, checkpointTime,
now);
+ activeRestoreTimeWindowedSum.record(metricsConfig,
activeRestoreTime, now);
+ standbyRestoreTimeWindowedSum.record(metricsConfig,
standbyRestoreTime, now);
+ }
+
+ private void recordRatios(final long now) {
+ final double idleTime = idleTimeWindowedSum.measure(metricsConfig,
now);
+ final double checkpointTime =
checkpointTimeWindowedSum.measure(metricsConfig, now);
+ final double activeRestoreTime =
activeRestoreTimeWindowedSum.measure(metricsConfig, now);
+ final double standbyRestoreTime =
standbyRestoreTimeWindowedSum.measure(metricsConfig, now);
+
+ final double totalTime = idleTime + checkpointTime +
activeRestoreTime + standbyRestoreTime;
Review Comment:
Does "total time" include the time the State Updater thread spends blocked
on the internal queue or waiting for tasks? Is this time covered by `idleTime`?
##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdater.java:
##########
@@ -713,19 +726,65 @@ private void measureCheckpointLatency(final Runnable
actionToMeasure) {
private void recordMetrics(final long now, final long totalLatency,
final long totalWaitLatency) {
final long totalRestoreLatency = Math.max(0L, totalLatency -
totalWaitLatency - totalCheckpointLatency);
- updaterMetrics.idleRatioSensor.record((double) totalWaitLatency /
totalLatency, now);
- updaterMetrics.checkpointRatioSensor.record((double)
totalCheckpointLatency / totalLatency, now);
+ recordWindowedSum(
+ now,
+ (double) totalWaitLatency,
+ (double) totalCheckpointLatency,
+ (double) totalRestoreLatency *
(changelogReader.isRestoringActive() ? 1.0d : 0.0d),
+ (double) totalRestoreLatency *
(changelogReader.isRestoringActive() ? 0.0d : 1.0d)
+ );
- if (changelogReader.isRestoringActive()) {
- updaterMetrics.activeRestoreRatioSensor.record((double)
totalRestoreLatency / totalLatency, now);
- updaterMetrics.standbyRestoreRatioSensor.record(0.0d, now);
- } else {
- updaterMetrics.standbyRestoreRatioSensor.record((double)
totalRestoreLatency / totalLatency, now);
- updaterMetrics.activeRestoreRatioSensor.record(0.0d, now);
- }
+ recordRatios(now);
totalCheckpointLatency = 0L;
}
+
+ private void initTimeWindowIfNeeded(final long now) {
+ if (!timeWindowInitialized) {
+ idleTimeWindowedSum.record(metricsConfig, 0.0, now);
+ checkpointTimeWindowedSum.record(metricsConfig, 0.0, now);
+ activeRestoreTimeWindowedSum.record(metricsConfig, 0.0, now);
+ standbyRestoreTimeWindowedSum.record(metricsConfig, 0.0, now);
+ timeWindowInitialized = true;
+ }
+ }
+
+ private void recordWindowedSum(final long now,
+ final double idleTime,
+ final double checkpointTime,
+ final double activeRestoreTime,
+ final double standbyRestoreTime) {
+ idleTimeWindowedSum.record(metricsConfig, idleTime, now);
+ checkpointTimeWindowedSum.record(metricsConfig, checkpointTime,
now);
+ activeRestoreTimeWindowedSum.record(metricsConfig,
activeRestoreTime, now);
+ standbyRestoreTimeWindowedSum.record(metricsConfig,
standbyRestoreTime, now);
+ }
+
+ private void recordRatios(final long now) {
+ final double idleTime = idleTimeWindowedSum.measure(metricsConfig,
now);
+ final double checkpointTime =
checkpointTimeWindowedSum.measure(metricsConfig, now);
+ final double activeRestoreTime =
activeRestoreTimeWindowedSum.measure(metricsConfig, now);
+ final double standbyRestoreTime =
standbyRestoreTimeWindowedSum.measure(metricsConfig, now);
+
+ final double totalTime = idleTime + checkpointTime +
activeRestoreTime + standbyRestoreTime;
Review Comment:
Why don't we use the total time as `totalTime = Math.max(0L, endTimeMs -
totalStartTimeMs);` as defined in `runOnce()`?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]