Skip to content

Commit 3738484

Browse files
Nikita-Shupletsovmjsax
authored andcommitted
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]>
1 parent 67a5af8 commit 3738484

File tree

2 files changed

+86
-1
lines changed

2 files changed

+86
-1
lines changed

streams/src/main/java/org/apache/kafka/streams/internals/metrics/OpenIterators.java

Lines changed: 13 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -23,8 +23,10 @@
2323
import org.apache.kafka.streams.state.internals.metrics.StateStoreMetrics;
2424

2525
import java.util.Comparator;
26+
import java.util.Iterator;
2627
import java.util.NavigableSet;
2728
import java.util.concurrent.ConcurrentSkipListSet;
29+
import java.util.concurrent.atomic.AtomicLong;
2830
import java.util.concurrent.atomic.LongAdder;
2931

3032
public class OpenIterators {
@@ -35,6 +37,7 @@ public class OpenIterators {
3537

3638
private final LongAdder numOpenIterators = new LongAdder();
3739
private final NavigableSet<MeteredIterator> openIterators = new ConcurrentSkipListSet<>(Comparator.comparingLong(MeteredIterator::startTimestamp));
40+
private final AtomicLong oldestStartTimestamp = new AtomicLong();
3841

3942
private MetricName metricName;
4043

@@ -51,10 +54,11 @@ public OpenIterators(final TaskId taskId,
5154
public void add(final MeteredIterator iterator) {
5255
openIterators.add(iterator);
5356
numOpenIterators.increment();
57+
updateOldestStartTimestamp();
5458

5559
if (numOpenIterators.intValue() == 1) {
5660
metricName = StateStoreMetrics.addOldestOpenIteratorGauge(taskId.toString(), metricsScope, name, streamsMetrics,
57-
(config, now) -> openIterators.first().startTimestamp()
61+
(config, now) -> oldestStartTimestamp.get()
5862
);
5963
}
6064
}
@@ -65,9 +69,17 @@ public void remove(final MeteredIterator iterator) {
6569
}
6670
numOpenIterators.decrement();
6771
openIterators.remove(iterator);
72+
updateOldestStartTimestamp();
6873
}
6974

7075
public long sum() {
7176
return numOpenIterators.sum();
7277
}
78+
79+
private void updateOldestStartTimestamp() {
80+
final Iterator<MeteredIterator> openIteratorsIterator = openIterators.iterator();
81+
if (openIteratorsIterator.hasNext()) {
82+
oldestStartTimestamp.set(openIteratorsIterator.next().startTimestamp());
83+
}
84+
}
7385
}
Lines changed: 73 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,73 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
package org.apache.kafka.streams.internals.metrics;
18+
19+
import org.apache.kafka.common.metrics.Gauge;
20+
import org.apache.kafka.streams.processor.TaskId;
21+
import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
22+
import org.apache.kafka.streams.state.internals.MeteredIterator;
23+
24+
import org.junit.jupiter.api.Test;
25+
import org.mockito.ArgumentCaptor;
26+
27+
import static org.hamcrest.CoreMatchers.is;
28+
import static org.hamcrest.CoreMatchers.not;
29+
import static org.hamcrest.MatcherAssert.assertThat;
30+
import static org.mockito.ArgumentMatchers.any;
31+
import static org.mockito.Mockito.mock;
32+
import static org.mockito.Mockito.never;
33+
import static org.mockito.Mockito.reset;
34+
import static org.mockito.Mockito.verify;
35+
36+
public class OpenIteratorsTest {
37+
38+
private final StreamsMetricsImpl streamsMetrics = mock(StreamsMetricsImpl.class);
39+
40+
@SuppressWarnings("unchecked")
41+
@Test
42+
public void shouldCalculateOldestStartTimestampCorrectly() {
43+
final OpenIterators openIterators = new OpenIterators(new TaskId(0, 0), "scope", "name", streamsMetrics);
44+
45+
final MeteredIterator meteredIterator1 = () -> 5;
46+
final MeteredIterator meteredIterator2 = () -> 2;
47+
final MeteredIterator meteredIterator3 = () -> 6;
48+
49+
openIterators.add(meteredIterator1);
50+
final ArgumentCaptor<Gauge<Long>> gaugeCaptor = ArgumentCaptor.forClass(Gauge.class);
51+
verify(streamsMetrics).addStoreLevelMutableMetric(any(), any(), any(), any(), any(), any(), gaugeCaptor.capture());
52+
final Gauge<Long> gauge = gaugeCaptor.getValue();
53+
assertThat(gauge.value(null, 0), is(5L));
54+
reset(streamsMetrics);
55+
56+
openIterators.add(meteredIterator2);
57+
verify(streamsMetrics, never()).addStoreLevelMutableMetric(any(), any(), any(), any(), any(), any(), gaugeCaptor.capture());
58+
assertThat(gauge.value(null, 0), is(2L));
59+
60+
openIterators.remove(meteredIterator2);
61+
verify(streamsMetrics, never()).removeMetric(any());
62+
assertThat(gauge.value(null, 0), is(5L));
63+
64+
openIterators.remove(meteredIterator1);
65+
verify(streamsMetrics).removeMetric(any());
66+
assertThat(gauge.value(null, 0), is(5L));
67+
68+
openIterators.add(meteredIterator3);
69+
verify(streamsMetrics).addStoreLevelMutableMetric(any(), any(), any(), any(), any(), any(), gaugeCaptor.capture());
70+
assertThat(gaugeCaptor.getValue(), not(gauge));
71+
assertThat(gaugeCaptor.getValue().value(null, 0), is(6L));
72+
}
73+
}

0 commit comments

Comments
 (0)