bbejeck commented on code in PR #20022:
URL: https://github.com/apache/kafka/pull/20022#discussion_r2165017062


##########
streams/src/main/java/org/apache/kafka/streams/internals/metrics/OpenIterators.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.internals.metrics;
+
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.streams.processor.TaskId;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.state.internals.MeteredIterator;
+import org.apache.kafka.streams.state.internals.metrics.StateStoreMetrics;
+
+import java.util.Comparator;
+import java.util.NavigableSet;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.LongAdder;
+
+public class OpenIterators {
+    private final TaskId taskId;
+    private final String metricsScope;
+    private final String name;
+    private final StreamsMetricsImpl streamsMetrics;
+
+    private final LongAdder numOpenIterators = new LongAdder();
+    private final NavigableSet<MeteredIterator> openIterators = new 
ConcurrentSkipListSet<>(Comparator.comparingLong(MeteredIterator::startTimestamp));
+
+    private MetricName metricName;
+
+    public OpenIterators(final TaskId taskId,
+                         final String metricsScope,
+                         final String name,
+                         final StreamsMetricsImpl streamsMetrics) {
+        this.taskId = taskId;
+        this.metricsScope = metricsScope;
+        this.name = name;
+        this.streamsMetrics = streamsMetrics;
+    }
+
+    public void add(final MeteredIterator iterator) {
+        openIterators.add(iterator);
+        numOpenIterators.increment();

Review Comment:
   Why not `openIterators.size()`?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/StateStoreMetrics.java:
##########
@@ -455,12 +456,12 @@ public static void addNumOpenIteratorsGauge(final String 
taskId,
 
     }
 
-    public static void addOldestOpenIteratorGauge(final String taskId,
-                                                  final String storeType,
-                                                  final String storeName,
-                                                  final StreamsMetricsImpl 
streamsMetrics,
-                                                  final Gauge<Long> 
oldestOpenIteratorGauge) {
-        streamsMetrics.addStoreLevelMutableMetric(
+    public static MetricName addOldestOpenIteratorGauge(final String taskId,
+                                                        final String storeType,
+                                                        final String storeName,
+                                                        final 
StreamsMetricsImpl streamsMetrics,
+                                                        final Gauge<Long> 
oldestOpenIteratorGauge) {
+        return streamsMetrics.addStoreLevelMutableMetric(

Review Comment:
   Same question here



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java:
##########
@@ -535,6 +539,8 @@ public <T> void addStoreLevelMutableMetric(final String 
taskId,
             final String key = 
storeSensorPrefix(Thread.currentThread().getName(), taskId, storeName);
             storeLevelMetrics.computeIfAbsent(key, ignored -> new 
LinkedList<>()).push(metricName);
         }
+
+        return metricName;

Review Comment:
   Why does `addX` return the `MetricName` obejct?



-- 
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: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to