Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,10 @@
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;

public class OpenIterators {
private final TaskId taskId;
Expand All @@ -33,6 +35,7 @@ public class OpenIterators {
private final StreamsMetricsImpl streamsMetrics;

private final NavigableSet<MeteredIterator> openIterators = new ConcurrentSkipListSet<>(Comparator.comparingLong(MeteredIterator::startTimestamp));
private final AtomicLong oldestStartTimestamp = new AtomicLong();

private MetricName metricName;

Expand All @@ -48,10 +51,11 @@ public OpenIterators(final TaskId taskId,

public void add(final MeteredIterator iterator) {
openIterators.add(iterator);
updateOldestStartTimestamp();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Possible nano-optimization: the oldest start timestamp will only change if this is the first open iterator - any other open iterator will definitionally be older - so we could possibly only update the oldest timestamp inside the if (openIterators.size() == 1) block below?
(assuming timestamps are all coming from the same time source)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am not 100% sure I understand the idea.
If we add 3 iterators with timestamps: 1, 2, 3
the result will be 1.
if we delete the first one, the result will be 2. the result changed, even though we still have more than 1 open iterator.

if I misunderstood the idea, please let me know

Copy link
Contributor

@stevenschlansker stevenschlansker Sep 10, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am only referring to the add() flow - which will add 1, then add 2, then add 3 - and only the first operation may change the oldest iterator.
I agree in the remove() case we must check every time.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

While I was reviewing the PR, I also though about this, but though it's not worth doing, as creating iterators should not happen with very high frequency?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In practice, I doubt it matters much, which is why I called it a "nano" optimization. I leave it in your capable hands to decide if it is worth implementing :)

That said, we actually are having scalability issues regarding creating iterators at relatively high frequency - see https://issues.apache.org/jira/browse/KAFKA-19678

One of our data stores is used almost exclusively for range lookups which use iterators.


if (openIterators.size() == 1) {
Copy link
Contributor

@stevenschlansker stevenschlansker Sep 10, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thinking about this more, I wonder if there is another possible race here that would not cause an exception but could cause missing or incorrectly persisting metrics. The OpenIterators tracker itself is not synchronized (good to avoid contention) but that means that we are mutating add() or remove() and calling size() expecting the results in program order.
But, I don't see any guarantee this is the case - my (limited) understanding is that OpenIterators is a property of a KeyValueStore, which may have partitions shared between threads.

Imagine:

T1: open iterator on store S1 (partition 0)
T2: open iterator on store S1 (partition 1)
T1: openIterators.add(it1)
T2: openIterators.add(it2)
T1: if (openIterators.size() == 1) {  (2 == 1? nope)
T2: if (openIterators.size() == 1) {  (2 == 1? nope)
End: 2 iterators open, no metric registered

Same race is possible with remove.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree. there is a potential race condition here. I will take a closer look

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think that's a problem. While stores are sharded, each shard is its own StateStore and thus has an own OpenIterator instance. Hence, OpenIterator should be used by a single StreamsThreads and there should not be any concurrent usage.

The concurrently issue we fixed in only for the metric, because addOldestOpenIteratorGauge registers the Gauge (config, now) -> openIterators.first().startTimestamp() (well, now (config, now) -> oldestStartTimestamp.get()) which could be used concurrently, when the metric is read.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK, thanks for clarifying!

metricName = StateStoreMetrics.addOldestOpenIteratorGauge(taskId.toString(), metricsScope, name, streamsMetrics,
(config, now) -> openIterators.first().startTimestamp()
(config, now) -> oldestStartTimestamp.get()
);
}
}
Expand All @@ -61,9 +65,17 @@ public void remove(final MeteredIterator iterator) {
streamsMetrics.removeMetric(metricName);
}
openIterators.remove(iterator);
updateOldestStartTimestamp();
}

public long sum() {
return openIterators.size();
}

private void updateOldestStartTimestamp() {
final Iterator<MeteredIterator> openIteratorsIterator = openIterators.iterator();
if (openIteratorsIterator.hasNext()) {
oldestStartTimestamp.set(openIteratorsIterator.next().startTimestamp());
}
}
}
Original file line number Diff line number Diff line change
@@ -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));
}
}