This is an automated email from the ASF dual-hosted git repository.
kamalcph pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git
The following commit(s) were added to refs/heads/trunk by this push:
new 7dba91d0251 KAFKA-19484: Fix bug with tiered storage throttle metrics
(#20129)
7dba91d0251 is described below
commit 7dba91d02513eee4ce090bd453f61960ad833065
Author: George Wu <[email protected]>
AuthorDate: Tue Jul 29 10:07:41 2025 -0400
KAFKA-19484: Fix bug with tiered storage throttle metrics (#20129)
Fixes a bug with tiered storage quota metrics introduced in
[KIP-956](https://cwiki.apache.org/confluence/display/KAFKA/KIP-956+Tiered+Storage+Quotas).
The metrics tracking how much time have been spent in a throttled state
can stop reporting if a cluster stops stops doing remote copy/fetch and
the sensors go inactive.
This change delegates the job of refreshing inactive sensors to
SensorAccess. There's pretty similar logic in RLMQuotaManager which is
actually responsible for tracking and enforcing quotas and also uses a
Sensor object.
```
remote-fetch-throttle-time-avg
remote-copy-throttle-time-avg
remote-fetch-throttle-time-max
remote-copy-throttle-time-max
```
Reviewers: Kamal Chandraprakash <[email protected]>
---
.../server/log/remote/quota/RLMQuotaMetrics.java | 25 ++++++++---
.../log/remote/storage/RemoteLogManager.java | 16 +++----
.../log/remote/quota/RLMQuotaMetricsTest.java | 52 ++++++++++++++++++++++
.../log/remote/storage/RemoteLogManagerTest.java | 10 ++++-
4 files changed, 86 insertions(+), 17 deletions(-)
diff --git
a/storage/src/main/java/org/apache/kafka/server/log/remote/quota/RLMQuotaMetrics.java
b/storage/src/main/java/org/apache/kafka/server/log/remote/quota/RLMQuotaMetrics.java
index bedcd1ae1a3..4e365e1a13d 100644
---
a/storage/src/main/java/org/apache/kafka/server/log/remote/quota/RLMQuotaMetrics.java
+++
b/storage/src/main/java/org/apache/kafka/server/log/remote/quota/RLMQuotaMetrics.java
@@ -26,18 +26,29 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
public class RLMQuotaMetrics {
- private final Sensor sensor;
+ private final SensorAccess sensorAccess;
+ private final Metrics metrics;
+ private final String name;
+ private final String descriptionFormat;
+ private final String group;
+ private final long expirationTime;
public RLMQuotaMetrics(Metrics metrics, String name, String group, String
descriptionFormat, long expirationTime) {
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
- SensorAccess sensorAccess = new SensorAccess(lock, metrics);
- this.sensor = sensorAccess.getOrCreate(name, expirationTime, s -> {
- s.add(metrics.metricName(name + "-avg", group,
String.format(descriptionFormat, "average")), new Avg());
- s.add(metrics.metricName(name + "-max", group,
String.format(descriptionFormat, "maximum")), new Max());
- });
+ this.sensorAccess = new SensorAccess(lock, metrics);
+ this.metrics = metrics;
+ this.name = name;
+ this.group = group;
+ this.expirationTime = expirationTime;
+ this.descriptionFormat = descriptionFormat;
}
public Sensor sensor() {
- return sensor;
+ return sensorAccess.getOrCreate(name, expirationTime, s -> {
+ s.add(metrics.metricName(name + "-avg", group,
+ String.format(descriptionFormat, "average")), new Avg());
+ s.add(metrics.metricName(name + "-max", group,
+ String.format(descriptionFormat, "maximum")), new Max());
+ });
}
}
diff --git
a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java
b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java
index 58bc519d3d1..692b348f73e 100644
---
a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java
+++
b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java
@@ -165,8 +165,8 @@ public class RemoteLogManager implements Closeable,
AsyncOffsetReader {
private final Condition copyQuotaManagerLockCondition =
copyQuotaManagerLock.newCondition();
private final RLMQuotaManager rlmCopyQuotaManager;
private final RLMQuotaManager rlmFetchQuotaManager;
- private final Sensor fetchThrottleTimeSensor;
- private final Sensor copyThrottleTimeSensor;
+ private final RLMQuotaMetrics fetchQuotaMetrics;
+ private final RLMQuotaMetrics copyQuotaMetrics;
private final RemoteIndexCache indexCache;
private final RemoteStorageThreadPool remoteStorageReaderThreadPool;
@@ -235,10 +235,10 @@ public class RemoteLogManager implements Closeable,
AsyncOffsetReader {
rlmCopyQuotaManager = createRLMCopyQuotaManager();
rlmFetchQuotaManager = createRLMFetchQuotaManager();
- fetchThrottleTimeSensor = new RLMQuotaMetrics(metrics,
"remote-fetch-throttle-time", RemoteLogManager.class.getSimpleName(),
- "The %s time in millis remote fetches was throttled by a broker",
INACTIVE_SENSOR_EXPIRATION_TIME_SECONDS).sensor();
- copyThrottleTimeSensor = new RLMQuotaMetrics(metrics,
"remote-copy-throttle-time", RemoteLogManager.class.getSimpleName(),
- "The %s time in millis remote copies was throttled by a broker",
INACTIVE_SENSOR_EXPIRATION_TIME_SECONDS).sensor();
+ fetchQuotaMetrics = new RLMQuotaMetrics(metrics,
"remote-fetch-throttle-time", RemoteLogManager.class.getSimpleName(),
+ "The %s time in millis remote fetches was throttled by a broker",
INACTIVE_SENSOR_EXPIRATION_TIME_SECONDS);
+ copyQuotaMetrics = new RLMQuotaMetrics(metrics,
"remote-copy-throttle-time", RemoteLogManager.class.getSimpleName(),
+ "The %s time in millis remote copies was throttled by a broker",
INACTIVE_SENSOR_EXPIRATION_TIME_SECONDS);
indexCache = new
RemoteIndexCache(rlmConfig.remoteLogIndexFileCacheTotalSizeBytes(),
remoteStorageManagerPlugin.get(), logDir);
delayInMs = rlmConfig.remoteLogManagerTaskIntervalMs();
@@ -347,7 +347,7 @@ public class RemoteLogManager implements Closeable,
AsyncOffsetReader {
}
public Sensor fetchThrottleTimeSensor() {
- return fetchThrottleTimeSensor;
+ return fetchQuotaMetrics.sensor();
}
static RLMQuotaManagerConfig copyQuotaManagerConfig(RemoteLogManagerConfig
rlmConfig) {
@@ -961,7 +961,7 @@ public class RemoteLogManager implements Closeable,
AsyncOffsetReader {
try {
long throttleTimeMs =
rlmCopyQuotaManager.getThrottleTimeMs();
while (throttleTimeMs > 0) {
-
copyThrottleTimeSensor.record(throttleTimeMs, time.milliseconds());
+
copyQuotaMetrics.sensor().record(throttleTimeMs, time.milliseconds());
logger.debug("Quota exceeded for copying
log segments, waiting for the quota to be available.");
// If the thread gets interrupted while
waiting, the InterruptedException is thrown
// back to the caller. It's important to
note that the task being executed is already
diff --git
a/storage/src/test/java/org/apache/kafka/server/log/remote/quota/RLMQuotaMetricsTest.java
b/storage/src/test/java/org/apache/kafka/server/log/remote/quota/RLMQuotaMetricsTest.java
new file mode 100644
index 00000000000..bf2cddd0f31
--- /dev/null
+++
b/storage/src/test/java/org/apache/kafka/server/log/remote/quota/RLMQuotaMetricsTest.java
@@ -0,0 +1,52 @@
+/*
+ * 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.server.log.remote.quota;
+
+import org.apache.kafka.common.metrics.MetricConfig;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.MockTime;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+
+public class RLMQuotaMetricsTest {
+ private final MockTime time = new MockTime();
+ private final Metrics metrics = new Metrics(new MetricConfig(), List.of(),
time);
+
+ @Test
+ public void testNewSensorWhenExpired() {
+ RLMQuotaMetrics rlmQuotaMetrics = new RLMQuotaMetrics(metrics,
"metric", "group", "format", 5);
+ Sensor sensor = rlmQuotaMetrics.sensor();
+ Sensor sensorRepeat = rlmQuotaMetrics.sensor();
+
+ // If the sensor has not expired we should reuse it.
+ assertEquals(sensorRepeat, sensor);
+
+ // The ExpireSensorTask calls removeSensor to remove expired sensors.
+ metrics.removeSensor(sensor.name());
+
+ // If the sensor has been removed, we should get a new one.
+ Sensor newSensor = rlmQuotaMetrics.sensor();
+ assertNotEquals(sensor, newSensor);
+ }
+}
diff --git
a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java
b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java
index c0b89bbf044..928688b55bb 100644
---
a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java
+++
b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java
@@ -3425,8 +3425,14 @@ public class RemoteLogManagerTest {
Map<org.apache.kafka.common.MetricName, KafkaMetric> allMetrics =
metrics.metrics();
KafkaMetric avgMetric =
allMetrics.get(metrics.metricName("remote-copy-throttle-time-avg",
"RemoteLogManager"));
KafkaMetric maxMetric =
allMetrics.get(metrics.metricName("remote-copy-throttle-time-max",
"RemoteLogManager"));
- assertEquals(Double.NaN, avgMetric.metricValue());
- assertEquals(Double.NaN, maxMetric.metricValue());
+ if (quotaExceeded) {
+ assertEquals(Double.NaN, avgMetric.metricValue());
+ assertEquals(Double.NaN, maxMetric.metricValue());
+ } else {
+ // Metrics are not created until they actually get recorded
(e.g. if the quota is exceeded).
+ assertNull(avgMetric);
+ assertNull(maxMetric);
+ }
// Verify the highest offset in remote storage is updated
ArgumentCaptor<Long> capture = ArgumentCaptor.forClass(Long.class);