This is an automated email from the ASF dual-hosted git repository.

mjsax pushed a commit to branch 3.6
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.6 by this push:
     new 4f1fdfe9864 KAFKA-13973: Fix inflated block cache metrics (#14317)
4f1fdfe9864 is described below

commit 4f1fdfe98648f0c7bd41d1897a419aafec2b9fe0
Author: Nick Telford <nick.telf...@gmail.com>
AuthorDate: Mon Sep 18 10:09:10 2023 +0100

    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 <matth...@confluent.io>, Bruno Cadonna 
<cado...@apache.org>
---
 .../state/internals/RocksDBTimestampedStore.java   |   2 +-
 .../internals/metrics/RocksDBMetricsRecorder.java  |   4 +-
 .../metrics/RocksDBBlockCacheMetricsTest.java      | 136 +++++++++++++++++++++
 .../metrics/RocksDBMetricsRecorderGaugesTest.java  |  19 ++-
 4 files changed, 155 insertions(+), 6 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java
index dd070597945..d2876c63f5b 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java
+++ 
b/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
 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);
     }
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorder.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorder.java
index df68f2e8024..7680b060802 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorder.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorder.java
@@ -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) {
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBBlockCacheMetricsTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBBlockCacheMetricsTest.java
new file mode 100644
index 00000000000..ecff033e037
--- /dev/null
+++ 
b/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBBlockCacheMetricsTest.java
@@ -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;
+    }
+}
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorderGaugesTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorderGaugesTest.java
index 9c21d1dba9e..a66116e733e 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorderGaugesTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorderGaugesTest.java
@@ -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 {
         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);
     }

Reply via email to