This is an automated email from the ASF dual-hosted git repository.
mjsax pushed a commit to branch 4.1
in repository https://gitbox.apache.org/repos/asf/kafka.git
The following commit(s) were added to refs/heads/4.1 by this push:
new 373848449aa KAFKA-19679: Fix NoSuchElementException in oldest open
iterator metric (#20512)
373848449aa is described below
commit 373848449aa26ccf6ef79b44a36ed0f59de1a10c
Author: Nikita Shupletsov <[email protected]>
AuthorDate: Tue Sep 9 21:18:00 2025 -0700
KAFKA-19679: Fix NoSuchElementException in oldest open iterator metric
(#20512)
Querying the oldest-open-iterator metric can result in a
NoSuchElementException when the last open iterator gets removed, due to
a race condition between the query and the metric update.
To avoid this race condition, this PR caches the metric result, to avoid
accessing the list of open iterator directly. We don't need to clear
this cache, because the entire metric is removed when the last iterator
gets removed.
Reviewers: Matthias J. Sax <[email protected]>
---
.../streams/internals/metrics/OpenIterators.java | 14 ++++-
.../internals/metrics/OpenIteratorsTest.java | 73 ++++++++++++++++++++++
2 files changed, 86 insertions(+), 1 deletion(-)
diff --git
a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/OpenIterators.java
b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/OpenIterators.java
index 5e2307d0ac6..2ef50d54abf 100644
---
a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/OpenIterators.java
+++
b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/OpenIterators.java
@@ -23,8 +23,10 @@ import
org.apache.kafka.streams.state.internals.MeteredIterator;
import org.apache.kafka.streams.state.internals.metrics.StateStoreMetrics;
import java.util.Comparator;
+import java.util.Iterator;
import java.util.NavigableSet;
import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.LongAdder;
public class OpenIterators {
@@ -35,6 +37,7 @@ public class OpenIterators {
private final LongAdder numOpenIterators = new LongAdder();
private final NavigableSet<MeteredIterator> openIterators = new
ConcurrentSkipListSet<>(Comparator.comparingLong(MeteredIterator::startTimestamp));
+ private final AtomicLong oldestStartTimestamp = new AtomicLong();
private MetricName metricName;
@@ -51,10 +54,11 @@ public class OpenIterators {
public void add(final MeteredIterator iterator) {
openIterators.add(iterator);
numOpenIterators.increment();
+ updateOldestStartTimestamp();
if (numOpenIterators.intValue() == 1) {
metricName =
StateStoreMetrics.addOldestOpenIteratorGauge(taskId.toString(), metricsScope,
name, streamsMetrics,
- (config, now) -> openIterators.first().startTimestamp()
+ (config, now) -> oldestStartTimestamp.get()
);
}
}
@@ -65,9 +69,17 @@ public class OpenIterators {
}
numOpenIterators.decrement();
openIterators.remove(iterator);
+ updateOldestStartTimestamp();
}
public long sum() {
return numOpenIterators.sum();
}
+
+ private void updateOldestStartTimestamp() {
+ final Iterator<MeteredIterator> openIteratorsIterator =
openIterators.iterator();
+ if (openIteratorsIterator.hasNext()) {
+
oldestStartTimestamp.set(openIteratorsIterator.next().startTimestamp());
+ }
+ }
}
diff --git
a/streams/src/test/java/org/apache/kafka/streams/internals/metrics/OpenIteratorsTest.java
b/streams/src/test/java/org/apache/kafka/streams/internals/metrics/OpenIteratorsTest.java
new file mode 100644
index 00000000000..3464ecbdaee
--- /dev/null
+++
b/streams/src/test/java/org/apache/kafka/streams/internals/metrics/OpenIteratorsTest.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.metrics.Gauge;
+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.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.not;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.verify;
+
+public class OpenIteratorsTest {
+
+ private final StreamsMetricsImpl streamsMetrics =
mock(StreamsMetricsImpl.class);
+
+ @SuppressWarnings("unchecked")
+ @Test
+ public void shouldCalculateOldestStartTimestampCorrectly() {
+ final OpenIterators openIterators = new OpenIterators(new TaskId(0,
0), "scope", "name", streamsMetrics);
+
+ final MeteredIterator meteredIterator1 = () -> 5;
+ final MeteredIterator meteredIterator2 = () -> 2;
+ final MeteredIterator meteredIterator3 = () -> 6;
+
+ openIterators.add(meteredIterator1);
+ final ArgumentCaptor<Gauge<Long>> gaugeCaptor =
ArgumentCaptor.forClass(Gauge.class);
+ verify(streamsMetrics).addStoreLevelMutableMetric(any(), any(), any(),
any(), any(), any(), gaugeCaptor.capture());
+ final Gauge<Long> gauge = gaugeCaptor.getValue();
+ assertThat(gauge.value(null, 0), is(5L));
+ reset(streamsMetrics);
+
+ openIterators.add(meteredIterator2);
+ verify(streamsMetrics, never()).addStoreLevelMutableMetric(any(),
any(), any(), any(), any(), any(), gaugeCaptor.capture());
+ assertThat(gauge.value(null, 0), is(2L));
+
+ openIterators.remove(meteredIterator2);
+ verify(streamsMetrics, never()).removeMetric(any());
+ assertThat(gauge.value(null, 0), is(5L));
+
+ openIterators.remove(meteredIterator1);
+ verify(streamsMetrics).removeMetric(any());
+ assertThat(gauge.value(null, 0), is(5L));
+
+ openIterators.add(meteredIterator3);
+ verify(streamsMetrics).addStoreLevelMutableMetric(any(), any(), any(),
any(), any(), any(), gaugeCaptor.capture());
+ assertThat(gaugeCaptor.getValue(), not(gauge));
+ assertThat(gaugeCaptor.getValue().value(null, 0), is(6L));
+ }
+}