mirror of https://github.com/apache/kafka.git
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 <kamal.chandraprakash@gmail.com>
This commit is contained in:
parent
4ff851a562
commit
7dba91d025
|
@ -26,18 +26,29 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||||
|
|
||||||
public class RLMQuotaMetrics {
|
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) {
|
public RLMQuotaMetrics(Metrics metrics, String name, String group, String descriptionFormat, long expirationTime) {
|
||||||
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
|
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
|
||||||
SensorAccess sensorAccess = new SensorAccess(lock, metrics);
|
this.sensorAccess = new SensorAccess(lock, metrics);
|
||||||
this.sensor = sensorAccess.getOrCreate(name, expirationTime, s -> {
|
this.metrics = metrics;
|
||||||
s.add(metrics.metricName(name + "-avg", group, String.format(descriptionFormat, "average")), new Avg());
|
this.name = name;
|
||||||
s.add(metrics.metricName(name + "-max", group, String.format(descriptionFormat, "maximum")), new Max());
|
this.group = group;
|
||||||
});
|
this.expirationTime = expirationTime;
|
||||||
|
this.descriptionFormat = descriptionFormat;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Sensor sensor() {
|
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());
|
||||||
|
});
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -165,8 +165,8 @@ public class RemoteLogManager implements Closeable, AsyncOffsetReader {
|
||||||
private final Condition copyQuotaManagerLockCondition = copyQuotaManagerLock.newCondition();
|
private final Condition copyQuotaManagerLockCondition = copyQuotaManagerLock.newCondition();
|
||||||
private final RLMQuotaManager rlmCopyQuotaManager;
|
private final RLMQuotaManager rlmCopyQuotaManager;
|
||||||
private final RLMQuotaManager rlmFetchQuotaManager;
|
private final RLMQuotaManager rlmFetchQuotaManager;
|
||||||
private final Sensor fetchThrottleTimeSensor;
|
private final RLMQuotaMetrics fetchQuotaMetrics;
|
||||||
private final Sensor copyThrottleTimeSensor;
|
private final RLMQuotaMetrics copyQuotaMetrics;
|
||||||
|
|
||||||
private final RemoteIndexCache indexCache;
|
private final RemoteIndexCache indexCache;
|
||||||
private final RemoteStorageThreadPool remoteStorageReaderThreadPool;
|
private final RemoteStorageThreadPool remoteStorageReaderThreadPool;
|
||||||
|
@ -235,10 +235,10 @@ public class RemoteLogManager implements Closeable, AsyncOffsetReader {
|
||||||
rlmCopyQuotaManager = createRLMCopyQuotaManager();
|
rlmCopyQuotaManager = createRLMCopyQuotaManager();
|
||||||
rlmFetchQuotaManager = createRLMFetchQuotaManager();
|
rlmFetchQuotaManager = createRLMFetchQuotaManager();
|
||||||
|
|
||||||
fetchThrottleTimeSensor = new RLMQuotaMetrics(metrics, "remote-fetch-throttle-time", RemoteLogManager.class.getSimpleName(),
|
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).sensor();
|
"The %s time in millis remote fetches was throttled by a broker", INACTIVE_SENSOR_EXPIRATION_TIME_SECONDS);
|
||||||
copyThrottleTimeSensor = new RLMQuotaMetrics(metrics, "remote-copy-throttle-time", RemoteLogManager.class.getSimpleName(),
|
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).sensor();
|
"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);
|
indexCache = new RemoteIndexCache(rlmConfig.remoteLogIndexFileCacheTotalSizeBytes(), remoteStorageManagerPlugin.get(), logDir);
|
||||||
delayInMs = rlmConfig.remoteLogManagerTaskIntervalMs();
|
delayInMs = rlmConfig.remoteLogManagerTaskIntervalMs();
|
||||||
|
@ -347,7 +347,7 @@ public class RemoteLogManager implements Closeable, AsyncOffsetReader {
|
||||||
}
|
}
|
||||||
|
|
||||||
public Sensor fetchThrottleTimeSensor() {
|
public Sensor fetchThrottleTimeSensor() {
|
||||||
return fetchThrottleTimeSensor;
|
return fetchQuotaMetrics.sensor();
|
||||||
}
|
}
|
||||||
|
|
||||||
static RLMQuotaManagerConfig copyQuotaManagerConfig(RemoteLogManagerConfig rlmConfig) {
|
static RLMQuotaManagerConfig copyQuotaManagerConfig(RemoteLogManagerConfig rlmConfig) {
|
||||||
|
@ -961,7 +961,7 @@ public class RemoteLogManager implements Closeable, AsyncOffsetReader {
|
||||||
try {
|
try {
|
||||||
long throttleTimeMs = rlmCopyQuotaManager.getThrottleTimeMs();
|
long throttleTimeMs = rlmCopyQuotaManager.getThrottleTimeMs();
|
||||||
while (throttleTimeMs > 0) {
|
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.");
|
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
|
// 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
|
// back to the caller. It's important to note that the task being executed is already
|
||||||
|
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
|
@ -3425,8 +3425,14 @@ public class RemoteLogManagerTest {
|
||||||
Map<org.apache.kafka.common.MetricName, KafkaMetric> allMetrics = metrics.metrics();
|
Map<org.apache.kafka.common.MetricName, KafkaMetric> allMetrics = metrics.metrics();
|
||||||
KafkaMetric avgMetric = allMetrics.get(metrics.metricName("remote-copy-throttle-time-avg", "RemoteLogManager"));
|
KafkaMetric avgMetric = allMetrics.get(metrics.metricName("remote-copy-throttle-time-avg", "RemoteLogManager"));
|
||||||
KafkaMetric maxMetric = allMetrics.get(metrics.metricName("remote-copy-throttle-time-max", "RemoteLogManager"));
|
KafkaMetric maxMetric = allMetrics.get(metrics.metricName("remote-copy-throttle-time-max", "RemoteLogManager"));
|
||||||
assertEquals(Double.NaN, avgMetric.metricValue());
|
if (quotaExceeded) {
|
||||||
assertEquals(Double.NaN, maxMetric.metricValue());
|
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
|
// Verify the highest offset in remote storage is updated
|
||||||
ArgumentCaptor<Long> capture = ArgumentCaptor.forClass(Long.class);
|
ArgumentCaptor<Long> capture = ArgumentCaptor.forClass(Long.class);
|
||||||
|
|
Loading…
Reference in New Issue