This is an automated email from the ASF dual-hosted git repository.
jeffkbkim pushed a commit to branch 4.0
in repository https://gitbox.apache.org/repos/asf/kafka.git
The following commit(s) were added to refs/heads/4.0 by this push:
new 58a7fab483e MINOR: prevent exception from HdrHistogram (#18674)
58a7fab483e is described below
commit 58a7fab483e5a2015ffa4686756acb9eda6aa6f2
Author: Jeff Kim <[email protected]>
AuthorDate: Wed Jan 29 11:34:46 2025 -0500
MINOR: prevent exception from HdrHistogram (#18674)
HdrHistogram can throw an exception if the recorded value is greater than a
configured limit. Expand the ceiling from per-metric to all invocations.
Reviewers: David Jacot <[email protected]>, Chia-Ping Tsai
<[email protected]>
---
.../runtime/CoordinatorRuntimeMetricsImpl.java | 4 +-
.../coordinator/common/runtime/HdrHistogram.java | 8 ++--
.../common/runtime/HdrHistogramTest.java | 9 ++++
.../common/runtime/KafkaMetricHistogramTest.java | 55 ++++++++++++++++++++++
4 files changed, 70 insertions(+), 6 deletions(-)
diff --git
a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeMetricsImpl.java
b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeMetricsImpl.java
index 391813250c1..591b37e2fb4 100644
---
a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeMetricsImpl.java
+++
b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeMetricsImpl.java
@@ -31,8 +31,6 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Supplier;
-import static
org.apache.kafka.coordinator.common.runtime.KafkaMetricHistogram.MAX_LATENCY_MS;
-
public class CoordinatorRuntimeMetricsImpl implements
CoordinatorRuntimeMetrics {
/**
@@ -293,7 +291,7 @@ public class CoordinatorRuntimeMetricsImpl implements
CoordinatorRuntimeMetrics
@Override
public void recordEventPurgatoryTime(long purgatoryTimeMs) {
- eventPurgatoryTimeSensor.record(Math.min(MAX_LATENCY_MS,
purgatoryTimeMs));
+ eventPurgatoryTimeSensor.record(purgatoryTimeMs);
}
@Override
diff --git
a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/HdrHistogram.java
b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/HdrHistogram.java
index 4b961d957cc..ac618430e93 100644
---
a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/HdrHistogram.java
+++
b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/HdrHistogram.java
@@ -49,6 +49,8 @@ public final class HdrHistogram {
*/
private final Recorder recorder;
+ private final long highestTrackableValue;
+
/**
* The latest snapshot of the internal HdrHistogram. Automatically updated
by
* {@link #latestHistogram(long)} if older than {@link #maxSnapshotAgeMs}.
@@ -69,6 +71,7 @@ public final class HdrHistogram {
) {
this.maxSnapshotAgeMs = maxSnapshotAgeMs;
recorder = new Recorder(highestTrackableValue,
numberOfSignificantValueDigits);
+ this.highestTrackableValue = highestTrackableValue;
this.timestampedHistogramSnapshot = new Timestamped<>(0, null);
}
@@ -89,13 +92,12 @@ public final class HdrHistogram {
}
/**
- * Writes to the histogram. Will throw {@link
ArrayIndexOutOfBoundsException} if the histogram's
- * highestTrackableValue is lower than the value being recorded.
+ * Writes to the histogram. Caps recording to highestTrackableValue
*
* @param value The value to be recorded. Cannot be negative.
*/
public void record(long value) {
- recorder.recordValue(value);
+ recorder.recordValue(Math.min(value, highestTrackableValue));
}
/**
diff --git
a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/HdrHistogramTest.java
b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/HdrHistogramTest.java
index 7703f11c81c..76e034ab70b 100644
---
a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/HdrHistogramTest.java
+++
b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/HdrHistogramTest.java
@@ -213,4 +213,13 @@ public class HdrHistogramTest {
}
ThreadUtils.shutdownExecutorServiceQuietly(countExecutor, 500,
TimeUnit.MILLISECONDS);
}
+
+ @Test
+ public void testRecordLimit() {
+ long highestTrackableValue = 10L;
+ HdrHistogram hdrHistogram = new HdrHistogram(10L,
highestTrackableValue, 3);
+
+ hdrHistogram.record(highestTrackableValue + 1000L);
+ assertEquals(highestTrackableValue,
hdrHistogram.max(System.currentTimeMillis()));
+ }
}
diff --git
a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/KafkaMetricHistogramTest.java
b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/KafkaMetricHistogramTest.java
new file mode 100644
index 00000000000..72a4bce3fae
--- /dev/null
+++
b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/KafkaMetricHistogramTest.java
@@ -0,0 +1,55 @@
+/*
+ * 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.coordinator.common.runtime;
+
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.metrics.CompoundStat;
+import org.apache.kafka.common.metrics.Metrics;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class KafkaMetricHistogramTest {
+
+ @Test
+ public void testStats() {
+ try (Metrics metrics = new Metrics()) {
+ KafkaMetricHistogram histogram =
KafkaMetricHistogram.newLatencyHistogram(
+ suffix -> metrics.metricName(
+ "test-metric-" + suffix,
+ "test-group",
+ "test description"
+ )
+ );
+
+ Set<MetricName> expected = Set.of(
+ new MetricName("test-metric-max", "test-group", "test
description", Collections.emptyMap()),
+ new MetricName("test-metric-p999", "test-group", "test
description", Collections.emptyMap()),
+ new MetricName("test-metric-p99", "test-group", "test
description", Collections.emptyMap()),
+ new MetricName("test-metric-p95", "test-group", "test
description", Collections.emptyMap()),
+ new MetricName("test-metric-p50", "test-group", "test
description", Collections.emptyMap())
+ );
+ Set<MetricName> actual =
histogram.stats().stream().map(CompoundStat.NamedMeasurable::name).collect(Collectors.toSet());
+ assertEquals(expected, actual);
+ }
+ }
+}