mjsax commented on code in PR #22267:
URL: https://github.com/apache/kafka/pull/22267#discussion_r3229322149


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java:
##########
@@ -182,7 +182,21 @@ private void registerMetrics() {
         );
         if (!persistent()) {
             StateStoreMetrics.addNumKeysGauge(taskId.toString(), metricsScope, 
name(), streamsMetrics,
-                    (config, now) -> wrapped().approximateNumEntries());
+                    (config, now) -> {
+                        final InMemoryKeyValueStore inMemoryStore = 
findInMemoryKeyValueStore(wrapped());
+                        return inMemoryStore != null ? 
inMemoryStore.approximateNumEntries() : -1L;
+                    }
+            );
+        }
+    }
+
+    private static InMemoryKeyValueStore findInMemoryKeyValueStore(final 
StateStore store) {
+        if (store instanceof InMemoryKeyValueStore) {
+            return (InMemoryKeyValueStore) store;
+        } else if (store instanceof WrappedStateStore) {
+            return findInMemoryKeyValueStore(((WrappedStateStore<?, ?, ?>) 
store).wrapped());
+        } else {
+            return null;

Review Comment:
   Should we maybe throw an exception here instead if returning `null`? We 
should never reach this point, and if we do, we have a bug? (If yes, should 
also be done for window/session case).



##########
streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/InMemoryStoreMetricsIntegrationTest.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.integration;
+
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
+import org.apache.kafka.streams.kstream.Consumed;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.kafka.streams.state.Stores;
+import org.apache.kafka.streams.state.internals.InMemoryKeyValueStore;
+import org.apache.kafka.test.MockApiProcessorSupplier;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.api.Timeout;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.CountDownLatch;
+
+import static org.apache.kafka.streams.utils.TestUtils.safeUniqueTestName;
+import static org.junit.jupiter.api.Assertions.fail;
+
+@Timeout(120)
+@Tag("integration")
+public class InMemoryStoreMetricsIntegrationTest {
+    private static final int NUM_BROKERS = 1;
+    private static final String INPUT_TOPIC = "in-memory-num-keys-input";
+
+    private final EmbeddedKafkaCluster cluster = new 
EmbeddedKafkaCluster(NUM_BROKERS);
+    private String safeTestName;
+
+    @BeforeEach
+    public void startCluster(final TestInfo testInfo) throws 
InterruptedException {
+        cluster.start();
+        cluster.createTopic(INPUT_TOPIC, 1, 1);
+        safeTestName = safeUniqueTestName(testInfo);
+    }
+
+    @AfterEach
+    public void closeCluster() {
+        cluster.stop();
+    }
+
+    @Test
+    public void metricValueShouldNotThrowIfStoreIsNotInitialized() throws 
Exception {
+        final CountDownLatch initLatch = new CountDownLatch(1);
+        final CountDownLatch finishLatch = new CountDownLatch(1);
+
+        final StreamsBuilder builder = new StreamsBuilder();
+        builder.addStateStore(
+            Stores.keyValueStoreBuilder(

Review Comment:
   Should we add (or does it already exist) similar ones for window and session 
case?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java:
##########
@@ -182,7 +182,21 @@ private void registerMetrics() {
         );
         if (!persistent()) {
             StateStoreMetrics.addNumKeysGauge(taskId.toString(), metricsScope, 
name(), streamsMetrics,
-                    (config, now) -> wrapped().approximateNumEntries());
+                    (config, now) -> {
+                        final InMemoryKeyValueStore inMemoryStore = 
findInMemoryKeyValueStore(wrapped());
+                        return inMemoryStore != null ? 
inMemoryStore.approximateNumEntries() : -1L;

Review Comment:
   Seem we have the same `null` check in window/session store, but wondering 
why we need it? If we get `null` back, would it not be a bug, and would mask 
the bug with the `null` check?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to