Browse Source

KAFKA-13973: Fix inflated block cache metrics (#14317)

All block cache metrics are being multiplied by the total number of
column families. In a `RocksDBTimestampedStore`, we have 2 column
families (the default, and the timestamped values), which causes all
block cache metrics in these stores to become doubled.

The cause is that our metrics recorder uses `getAggregatedLongProperty`
to fetch block cache metrics. `getAggregatedLongProperty` queries the
property on each column family in the database, and sums the results.

Since we always configure all column families to share the same block
cache, that causes the same block cache to be queried multiple times for
its metrics, with the results added togehter, effectively multiplying
the real value by the total number of column families.

To fix this, we should simply use `getLongProperty`, which queries a
single column family (the default one). Since all column families share
the same block cache, querying just one of them will give us the correct
metrics for that shared block cache.

Note: the same block cache is shared among all column families of a store
irrespective of whether the user has configured a shared block cache
across multiple stores.

Reviewers: Matthias J. Sax <matthias@confluent.io>, Bruno Cadonna <cadonna@apache.org>
pull/14395/head
Nick Telford 1 year ago committed by GitHub
parent
commit
f041efa5fd
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 2
      streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java
  2. 4
      streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorder.java
  3. 136
      streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBBlockCacheMetricsTest.java
  4. 19
      streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorderGaugesTest.java

2
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java

@ -50,7 +50,7 @@ import static org.apache.kafka.streams.state.TimestampedBytesStore.convertToTime @@ -50,7 +50,7 @@ import static org.apache.kafka.streams.state.TimestampedBytesStore.convertToTime
public class RocksDBTimestampedStore extends RocksDBStore implements TimestampedBytesStore {
private static final Logger log = LoggerFactory.getLogger(RocksDBTimestampedStore.class);
RocksDBTimestampedStore(final String name,
public RocksDBTimestampedStore(final String name,
final String metricsScope) {
super(name, metricsScope);
}

4
streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorder.java

@ -373,14 +373,14 @@ public class RocksDBMetricsRecorder { @@ -373,14 +373,14 @@ public class RocksDBMetricsRecorder {
// values of RocksDB properties are of type unsigned long in C++, i.e., in Java we need to use
// BigInteger and construct the object from the byte representation of the value
result = new BigInteger(1, longToBytes(
valueProvider.db.getAggregatedLongProperty(ROCKSDB_PROPERTIES_PREFIX + propertyName)
valueProvider.db.getLongProperty(ROCKSDB_PROPERTIES_PREFIX + propertyName)
));
break;
} else {
// values of RocksDB properties are of type unsigned long in C++, i.e., in Java we need to use
// BigInteger and construct the object from the byte representation of the value
result = result.add(new BigInteger(1, longToBytes(
valueProvider.db.getAggregatedLongProperty(ROCKSDB_PROPERTIES_PREFIX + propertyName)
valueProvider.db.getLongProperty(ROCKSDB_PROPERTIES_PREFIX + propertyName)
)));
}
} catch (final RocksDBException e) {

136
streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBBlockCacheMetricsTest.java

@ -0,0 +1,136 @@ @@ -0,0 +1,136 @@
/*
* 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.streams.state.internals.metrics;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.metrics.KafkaMetric;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.streams.processor.StateStoreContext;
import org.apache.kafka.streams.processor.TaskId;
import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
import org.apache.kafka.streams.state.internals.BlockBasedTableConfigWithAccessibleCache;
import org.apache.kafka.streams.state.internals.RocksDBStore;
import org.apache.kafka.streams.state.internals.RocksDBTimestampedStore;
import org.apache.kafka.test.MockInternalProcessorContext;
import org.apache.kafka.test.TestUtils;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
import java.io.File;
import java.io.IOException;
import java.math.BigInteger;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Properties;
import java.util.stream.Stream;
import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.STATE_STORE_LEVEL_GROUP;
import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.STORE_ID_TAG;
import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.TASK_ID_TAG;
import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.THREAD_ID_TAG;
import static org.junit.jupiter.api.Assertions.assertEquals;
public class RocksDBBlockCacheMetricsTest {
private static final String STORE_NAME = "test";
private static final String METRICS_SCOPE = "test-scope";
private static TaskId taskId = new TaskId(0, 0);
public static Stream<Arguments> stores() {
final File stateDir = TestUtils.tempDirectory("state");
return Stream.of(
Arguments.of(new RocksDBStore(STORE_NAME, METRICS_SCOPE), new MockInternalProcessorContext(new Properties(), taskId, stateDir)),
Arguments.of(new RocksDBTimestampedStore(STORE_NAME, METRICS_SCOPE), new MockInternalProcessorContext(new Properties(), taskId, stateDir))
);
}
static void withStore(final RocksDBStore store, final StateStoreContext context, final Runnable function) {
store.init(context, store);
try {
function.run();
} finally {
store.close();
try {
Utils.delete(context.stateDir());
} catch (final IOException e) {
// ignore
}
}
}
@ParameterizedTest
@MethodSource("stores")
public void shouldRecordCorrectBlockCacheCapacity(final RocksDBStore store, final StateStoreContext ctx) {
withStore(store, ctx, () ->
assertMetric(ctx, STATE_STORE_LEVEL_GROUP, RocksDBMetrics.CAPACITY_OF_BLOCK_CACHE, BigInteger.valueOf(50 * 1024 * 1024L)));
}
@ParameterizedTest
@MethodSource("stores")
public void shouldRecordCorrectBlockCacheUsage(final RocksDBStore store, final StateStoreContext ctx) {
withStore(store, ctx, () -> {
final BlockBasedTableConfigWithAccessibleCache tableFormatConfig = (BlockBasedTableConfigWithAccessibleCache) store.getOptions().tableFormatConfig();
final long usage = tableFormatConfig.blockCache().getUsage();
assertMetric(ctx, STATE_STORE_LEVEL_GROUP, RocksDBMetrics.USAGE_OF_BLOCK_CACHE, BigInteger.valueOf(usage));
});
}
@ParameterizedTest
@MethodSource("stores")
public void shouldRecordCorrectBlockCachePinnedUsage(final RocksDBStore store, final StateStoreContext ctx) {
withStore(store, ctx, () -> {
final BlockBasedTableConfigWithAccessibleCache tableFormatConfig = (BlockBasedTableConfigWithAccessibleCache) store.getOptions().tableFormatConfig();
final long usage = tableFormatConfig.blockCache().getPinnedUsage();
assertMetric(ctx, STATE_STORE_LEVEL_GROUP, RocksDBMetrics.PINNED_USAGE_OF_BLOCK_CACHE, BigInteger.valueOf(usage));
});
}
public <T> void assertMetric(final StateStoreContext context, final String group, final String metricName, final T expected) {
final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
final MetricName name = metrics.metricsRegistry().metricName(
metricName,
group,
"Ignored",
storeLevelTagMap(taskId.toString(), METRICS_SCOPE, STORE_NAME)
);
final KafkaMetric metric = (KafkaMetric) metrics.metrics().get(name);
assertEquals(expected, metric.metricValue(), String.format("Value for metric '%s-%s' was incorrect", group, metricName));
}
public Map<String, String> threadLevelTagMap(final String threadId) {
final Map<String, String> tagMap = new LinkedHashMap<>();
tagMap.put(THREAD_ID_TAG, threadId);
return tagMap;
}
public Map<String, String> taskLevelTagMap(final String threadId, final String taskId) {
final Map<String, String> tagMap = threadLevelTagMap(threadId);
tagMap.put(TASK_ID_TAG, taskId);
return tagMap;
}
public Map<String, String> storeLevelTagMap(final String taskName,
final String storeType,
final String storeName) {
final Map<String, String> tagMap = taskLevelTagMap(Thread.currentThread().getName(), taskName);
tagMap.put(storeType + "-" + STORE_ID_TAG, storeName);
return tagMap;
}
}

19
streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorderGaugesTest.java

@ -218,7 +218,20 @@ public class RocksDBMetricsRecorderGaugesTest { @@ -218,7 +218,20 @@ public class RocksDBMetricsRecorderGaugesTest {
}
private void runAndVerifyBlockCacheMetricsWithMultipleCaches(final String propertyName) throws Exception {
runAndVerifySumOfProperties(propertyName);
final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(new Metrics(), "test-client", StreamsConfig.METRICS_LATEST, new MockTime());
final RocksDBMetricsRecorder recorder = new RocksDBMetricsRecorder(METRICS_SCOPE, STORE_NAME);
recorder.init(streamsMetrics, TASK_ID);
recorder.addValueProviders(SEGMENT_STORE_NAME_1, dbToAdd1, cacheToAdd1, statisticsToAdd1);
recorder.addValueProviders(SEGMENT_STORE_NAME_2, dbToAdd2, cacheToAdd2, statisticsToAdd2);
final long recordedValue1 = 5L;
final long recordedValue2 = 3L;
when(dbToAdd1.getLongProperty(ROCKSDB_PROPERTIES_PREFIX + propertyName)).thenReturn(recordedValue1);
when(dbToAdd2.getLongProperty(ROCKSDB_PROPERTIES_PREFIX + propertyName)).thenReturn(recordedValue2);
verifyMetrics(streamsMetrics, propertyName, recordedValue1 + recordedValue2);
}
private void runAndVerifyBlockCacheMetricsWithSingleCache(final String propertyName) throws Exception {
@ -231,8 +244,8 @@ public class RocksDBMetricsRecorderGaugesTest { @@ -231,8 +244,8 @@ public class RocksDBMetricsRecorderGaugesTest {
recorder.addValueProviders(SEGMENT_STORE_NAME_2, dbToAdd2, cacheToAdd1, statisticsToAdd2);
final long recordedValue = 5L;
when(dbToAdd1.getAggregatedLongProperty(ROCKSDB_PROPERTIES_PREFIX + propertyName)).thenReturn(recordedValue);
when(dbToAdd2.getAggregatedLongProperty(ROCKSDB_PROPERTIES_PREFIX + propertyName)).thenReturn(recordedValue);
when(dbToAdd1.getLongProperty(ROCKSDB_PROPERTIES_PREFIX + propertyName)).thenReturn(recordedValue);
when(dbToAdd2.getLongProperty(ROCKSDB_PROPERTIES_PREFIX + propertyName)).thenReturn(recordedValue);
verifyMetrics(streamsMetrics, propertyName, recordedValue);
}

Loading…
Cancel
Save