Browse Source

KAFKA-14664; Fix inaccurate raft idle ratio metric (#13207)

The raft idle ratio is currently computed as the average of all recorded poll durations. This tends to underestimate the actual idle ratio since it treats all measurements equally regardless how much time was spent. For example, say we poll twice with the following durations:

Poll 1: 2s
Poll 2: 0s

Assume that the busy time is negligible, so 2s passes overall.

In the first measurement, 2s is spent waiting, so we compute and record a ratio of 1.0. In the second measurement, no time passes, and we record 0.0. The idle ratio is then computed as the average of these two values (1.0 + 0.0 / 2 = 0.5), which suggests that the process was busy for 1s, which overestimates the true busy time.

In this patch, we create a new `TimeRatio` class which tracks the total duration of a periodic event over a full interval of time measurement.

Reviewers: José Armando García Sancio <jsancio@apache.org>
pull/13192/head
Jason Gustafson 2 years ago committed by GitHub
parent
commit
35142d43e6
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 21
      raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
  2. 32
      raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java
  3. 82
      raft/src/main/java/org/apache/kafka/raft/internals/TimeRatio.java
  4. 60
      raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java
  5. 67
      raft/src/test/java/org/apache/kafka/raft/internals/TimeRatioTest.java

21
raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java

@ -2224,27 +2224,28 @@ public class KafkaRaftClient<T> implements RaftClient<T> { @@ -2224,27 +2224,28 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
* requests and send any needed outbound requests.
*/
public void poll() {
pollListeners();
long currentTimeMs = time.milliseconds();
if (maybeCompleteShutdown(currentTimeMs)) {
long startPollTimeMs = time.milliseconds();
if (maybeCompleteShutdown(startPollTimeMs)) {
return;
}
long pollStateTimeoutMs = pollCurrentState(currentTimeMs);
long cleaningTimeoutMs = snapshotCleaner.maybeClean(currentTimeMs);
long pollStateTimeoutMs = pollCurrentState(startPollTimeMs);
long cleaningTimeoutMs = snapshotCleaner.maybeClean(startPollTimeMs);
long pollTimeoutMs = Math.min(pollStateTimeoutMs, cleaningTimeoutMs);
kafkaRaftMetrics.updatePollStart(currentTimeMs);
long startWaitTimeMs = time.milliseconds();
kafkaRaftMetrics.updatePollStart(startWaitTimeMs);
RaftMessage message = messageQueue.poll(pollTimeoutMs);
currentTimeMs = time.milliseconds();
kafkaRaftMetrics.updatePollEnd(currentTimeMs);
long endWaitTimeMs = time.milliseconds();
kafkaRaftMetrics.updatePollEnd(endWaitTimeMs);
if (message != null) {
handleInboundMessage(message, currentTimeMs);
handleInboundMessage(message, endWaitTimeMs);
}
pollListeners();
}
@Override

32
raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java

@ -39,7 +39,6 @@ public class KafkaRaftMetrics implements AutoCloseable { @@ -39,7 +39,6 @@ public class KafkaRaftMetrics implements AutoCloseable {
private volatile int numUnknownVoterConnections;
private volatile OptionalLong electionStartMs;
private volatile OptionalLong pollStartMs;
private volatile OptionalLong pollEndMs;
private final MetricName currentLeaderIdMetricName;
private final MetricName currentVotedIdMetricName;
@ -53,14 +52,13 @@ public class KafkaRaftMetrics implements AutoCloseable { @@ -53,14 +52,13 @@ public class KafkaRaftMetrics implements AutoCloseable {
private final Sensor electionTimeSensor;
private final Sensor fetchRecordsSensor;
private final Sensor appendRecordsSensor;
private final Sensor pollIdleSensor;
private final Sensor pollDurationSensor;
public KafkaRaftMetrics(Metrics metrics, String metricGrpPrefix, QuorumState state) {
this.metrics = metrics;
String metricGroupName = metricGrpPrefix + "-metrics";
this.pollStartMs = OptionalLong.empty();
this.pollEndMs = OptionalLong.empty();
this.electionStartMs = OptionalLong.empty();
this.numUnknownVoterConnections = 0;
this.logEndOffset = new OffsetAndEpoch(0L, 0);
@ -133,26 +131,28 @@ public class KafkaRaftMetrics implements AutoCloseable { @@ -133,26 +131,28 @@ public class KafkaRaftMetrics implements AutoCloseable {
"The average number of records appended per sec as the leader of the raft quorum."),
new Rate(TimeUnit.SECONDS, new WindowedSum()));
this.pollIdleSensor = metrics.sensor("poll-idle-ratio");
this.pollIdleSensor.add(metrics.metricName("poll-idle-ratio-avg",
this.pollDurationSensor = metrics.sensor("poll-idle-ratio");
this.pollDurationSensor.add(
metrics.metricName(
"poll-idle-ratio-avg",
metricGroupName,
"The average fraction of time the client's poll() is idle as opposed to waiting for the user code to process records."),
new Avg());
"The ratio of time the Raft IO thread is idle as opposed to " +
"doing work (e.g. handling requests or replicating from the leader)"
),
new TimeRatio(1.0)
);
}
public void updatePollStart(long currentTimeMs) {
if (pollEndMs.isPresent() && pollStartMs.isPresent()) {
long pollTimeMs = Math.max(pollEndMs.getAsLong() - pollStartMs.getAsLong(), 0L);
long totalTimeMs = Math.max(currentTimeMs - pollStartMs.getAsLong(), 1L);
this.pollIdleSensor.record(pollTimeMs / (double) totalTimeMs, currentTimeMs);
}
this.pollStartMs = OptionalLong.of(currentTimeMs);
this.pollEndMs = OptionalLong.empty();
}
public void updatePollEnd(long currentTimeMs) {
this.pollEndMs = OptionalLong.of(currentTimeMs);
if (pollStartMs.isPresent()) {
long pollDurationMs = Math.max(currentTimeMs - pollStartMs.getAsLong(), 0L);
this.pollDurationSensor.record(pollDurationMs);
this.pollStartMs = OptionalLong.empty();
}
}
public void updateLogEnd(OffsetAndEpoch logEndOffset) {
@ -204,7 +204,7 @@ public class KafkaRaftMetrics implements AutoCloseable { @@ -204,7 +204,7 @@ public class KafkaRaftMetrics implements AutoCloseable {
electionTimeSensor.name(),
fetchRecordsSensor.name(),
appendRecordsSensor.name(),
pollIdleSensor.name()
pollDurationSensor.name()
).forEach(metrics::removeSensor);
}
}

82
raft/src/main/java/org/apache/kafka/raft/internals/TimeRatio.java

@ -0,0 +1,82 @@ @@ -0,0 +1,82 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.raft.internals;
import org.apache.kafka.common.metrics.MeasurableStat;
import org.apache.kafka.common.metrics.MetricConfig;
/**
* Maintains an approximate ratio of the duration of a specific event
* over all time. For example, this can be used to compute the ratio of
* time that a thread is busy or idle. The value is approximate since the
* measurement and recording intervals may not be aligned.
*
* Note that the duration of the event is assumed to be small relative to
* the interval of measurement.
*
*/
public class TimeRatio implements MeasurableStat {
private long intervalStartTimestampMs = -1;
private long lastRecordedTimestampMs = -1;
private double totalRecordedDurationMs = 0;
private final double defaultRatio;
public TimeRatio(double defaultRatio) {
if (defaultRatio < 0.0 || defaultRatio > 1.0) {
throw new IllegalArgumentException("Invalid ratio: value " + defaultRatio + " is not between 0 and 1.");
}
this.defaultRatio = defaultRatio;
}
@Override
public double measure(MetricConfig config, long currentTimestampMs) {
if (lastRecordedTimestampMs < 0) {
// Return the default value if no recordings have been captured.
return defaultRatio;
} else {
// We measure the ratio over the
double intervalDurationMs = Math.max(lastRecordedTimestampMs - intervalStartTimestampMs, 0);
final double ratio;
if (intervalDurationMs == 0) {
ratio = defaultRatio;
} else if (totalRecordedDurationMs > intervalDurationMs) {
ratio = 1.0;
} else {
ratio = totalRecordedDurationMs / intervalDurationMs;
}
// The next interval begins at the
intervalStartTimestampMs = lastRecordedTimestampMs;
totalRecordedDurationMs = 0;
return ratio;
}
}
@Override
public void record(MetricConfig config, double value, long currentTimestampMs) {
if (intervalStartTimestampMs < 0) {
// Discard the initial value since the value occurred prior to the interval start
intervalStartTimestampMs = currentTimestampMs;
} else {
totalRecordedDurationMs += value;
lastRecordedTimestampMs = currentTimestampMs;
}
}
}

60
raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java

@ -190,25 +190,75 @@ public class KafkaRaftMetricsTest { @@ -190,25 +190,75 @@ public class KafkaRaftMetricsTest {
}
@Test
public void shouldRecordPollIdleRatio() throws IOException {
public void shouldRecordPollIdleRatio() {
QuorumState state = buildQuorumState(Collections.singleton(localId));
state.initialize(new OffsetAndEpoch(0L, 0));
raftMetrics = new KafkaRaftMetrics(metrics, "raft", state);
// First recording is discarded (in order to align the interval of measurement)
raftMetrics.updatePollStart(time.milliseconds());
raftMetrics.updatePollEnd(time.milliseconds());
// Idle for 100ms
raftMetrics.updatePollStart(time.milliseconds());
time.sleep(100L);
raftMetrics.updatePollEnd(time.milliseconds());
time.sleep(900L);
// Busy for 100ms
time.sleep(100L);
// Idle for 200ms
raftMetrics.updatePollStart(time.milliseconds());
time.sleep(200L);
raftMetrics.updatePollEnd(time.milliseconds());
assertEquals(0.1, getMetric(metrics, "poll-idle-ratio-avg").metricValue());
assertEquals(0.75, getMetric(metrics, "poll-idle-ratio-avg").metricValue());
// Busy for 100ms
time.sleep(100L);
// Idle for 75ms
raftMetrics.updatePollStart(time.milliseconds());
time.sleep(75L);
raftMetrics.updatePollEnd(time.milliseconds());
time.sleep(100L);
// Idle for 25ms
raftMetrics.updatePollStart(time.milliseconds());
time.sleep(25L);
raftMetrics.updatePollEnd(time.milliseconds());
// Idle for 0ms
raftMetrics.updatePollStart(time.milliseconds());
raftMetrics.updatePollEnd(time.milliseconds());
assertEquals(0.5, getMetric(metrics, "poll-idle-ratio-avg").metricValue());
// Busy for 40ms
time.sleep(40);
// Idle for 60ms
raftMetrics.updatePollStart(time.milliseconds());
time.sleep(60);
raftMetrics.updatePollEnd(time.milliseconds());
// Busy for 10ms
time.sleep(10);
// Begin idle time for 5ms
raftMetrics.updatePollStart(time.milliseconds());
time.sleep(5);
// Measurement arrives before poll end, so we have 40ms busy time and 60ms idle.
// The subsequent interval time is not counted until the next measurement.
assertEquals(0.6, getMetric(metrics, "poll-idle-ratio-avg").metricValue());
// More idle time for 5ms
time.sleep(5);
raftMetrics.updatePollEnd(time.milliseconds());
assertEquals(0.3, getMetric(metrics, "poll-idle-ratio-avg").metricValue());
// The measurement includes the interval beginning at the last recording.
// This counts 10ms of busy time and 5ms + 5ms = 10ms of idle time.
assertEquals(0.5, getMetric(metrics, "poll-idle-ratio-avg").metricValue());
}
@Test

67
raft/src/test/java/org/apache/kafka/raft/internals/TimeRatioTest.java

@ -0,0 +1,67 @@ @@ -0,0 +1,67 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.raft.internals;
import org.apache.kafka.common.metrics.MetricConfig;
import org.apache.kafka.common.utils.MockTime;
import org.junit.jupiter.api.Test;
import static org.junit.jupiter.api.Assertions.assertEquals;
class TimeRatioTest {
@Test
public void testRatio() {
MetricConfig config = new MetricConfig();
MockTime time = new MockTime();
TimeRatio ratio = new TimeRatio(1.0);
ratio.record(config, 0.0, time.milliseconds());
time.sleep(10);
ratio.record(config, 10, time.milliseconds());
time.sleep(10);
ratio.record(config, 0, time.milliseconds());
assertEquals(0.5, ratio.measure(config, time.milliseconds()));
time.sleep(10);
ratio.record(config, 10, time.milliseconds());
time.sleep(40);
ratio.record(config, 0, time.milliseconds());
assertEquals(0.2, ratio.measure(config, time.milliseconds()));
}
@Test
public void testRatioMisalignedWindow() {
MetricConfig config = new MetricConfig();
MockTime time = new MockTime();
TimeRatio ratio = new TimeRatio(1.0);
ratio.record(config, 0.0, time.milliseconds());
time.sleep(10);
ratio.record(config, 10, time.milliseconds());
time.sleep(10);
// No recordings, so the last 10ms are not counted.
assertEquals(1.0, ratio.measure(config, time.milliseconds()));
// Now the measurement of 5ms arrives. We measure the time since the last
// recording, so 5ms/10ms = 0.5.
ratio.record(config, 5, time.milliseconds());
assertEquals(0.5, ratio.measure(config, time.milliseconds()));
}
}
Loading…
Cancel
Save