Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-15541: Use LongAdder instead of AtomicInteger #16076

Merged
merged 1 commit into from
May 25, 2024
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 @@ -52,7 +52,7 @@
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.LongAdder;
import java.util.function.Function;

import static org.apache.kafka.common.utils.Utils.mkEntry;
Expand Down Expand Up @@ -95,7 +95,7 @@ public class MeteredKeyValueStore<K, V>
private StreamsMetricsImpl streamsMetrics;
private TaskId taskId;

protected AtomicInteger numOpenIterators = new AtomicInteger(0);
protected LongAdder numOpenIterators = new LongAdder();

@SuppressWarnings("rawtypes")
private final Map<Class, QueryHandler> queryHandlers =
Expand Down Expand Up @@ -168,7 +168,7 @@ private void registerMetrics() {
e2eLatencySensor = StateStoreMetrics.e2ELatencySensor(taskId.toString(), metricsScope, name(), streamsMetrics);
iteratorDurationSensor = StateStoreMetrics.iteratorDurationSensor(taskId.toString(), metricsScope, name(), streamsMetrics);
StateStoreMetrics.addNumOpenIteratorsGauge(taskId.toString(), metricsScope, name(), streamsMetrics,
(config, now) -> numOpenIterators.get());
(config, now) -> numOpenIterators.sum());
}

protected Serde<V> prepareValueSerdeForStore(final Serde<V> valueSerde, final SerdeGetter getter) {
Expand Down Expand Up @@ -467,7 +467,7 @@ private MeteredKeyValueIterator(final KeyValueIterator<Bytes, byte[]> iter,
this.iter = iter;
this.sensor = sensor;
this.startNs = time.nanoseconds();
numOpenIterators.incrementAndGet();
numOpenIterators.increment();
}

@Override
Expand All @@ -491,7 +491,7 @@ public void close() {
final long duration = time.nanoseconds() - startNs;
sensor.record(duration);
iteratorDurationSensor.record(duration);
numOpenIterators.decrementAndGet();
numOpenIterators.decrement();
}
}

Expand All @@ -515,7 +515,7 @@ private MeteredKeyValueTimestampedIterator(final KeyValueIterator<Bytes, byte[]>
this.sensor = sensor;
this.valueDeserializer = valueDeserializer;
this.startNs = time.nanoseconds();
numOpenIterators.incrementAndGet();
numOpenIterators.increment();
}

@Override
Expand All @@ -539,7 +539,7 @@ public void close() {
final long duration = time.nanoseconds() - startNs;
sensor.record(duration);
iteratorDurationSensor.record(duration);
numOpenIterators.decrementAndGet();
numOpenIterators.decrement();
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
*/
package org.apache.kafka.streams.state.internals;

import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.LongAdder;
import java.util.function.Function;

import org.apache.kafka.common.metrics.Sensor;
Expand All @@ -28,7 +28,7 @@ public class MeteredMultiVersionedKeyQueryIterator<V> implements VersionedRecord

private final VersionedRecordIterator<byte[]> iterator;
private final Function<VersionedRecord<byte[]>, VersionedRecord<V>> deserializeValue;
private final AtomicInteger numOpenIterators;
private final LongAdder numOpenIterators;
private final Sensor sensor;
private final Time time;
private final long startNs;
Expand All @@ -37,14 +37,14 @@ public MeteredMultiVersionedKeyQueryIterator(final VersionedRecordIterator<byte[
final Sensor sensor,
final Time time,
final Function<VersionedRecord<byte[]>, VersionedRecord<V>> deserializeValue,
final AtomicInteger numOpenIterators) {
final LongAdder numOpenIterators) {
this.iterator = iterator;
this.deserializeValue = deserializeValue;
this.numOpenIterators = numOpenIterators;
this.sensor = sensor;
this.time = time;
this.startNs = time.nanoseconds();
numOpenIterators.incrementAndGet();
numOpenIterators.increment();
}


Expand All @@ -54,7 +54,7 @@ public void close() {
iterator.close();
} finally {
sensor.record(time.nanoseconds() - startNs);
numOpenIterators.decrementAndGet();
numOpenIterators.decrement();
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@

import java.util.Map;
import java.util.Objects;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.LongAdder;

import static org.apache.kafka.common.utils.Utils.mkEntry;
import static org.apache.kafka.common.utils.Utils.mkMap;
Expand All @@ -72,7 +72,7 @@ public class MeteredSessionStore<K, V>
private InternalProcessorContext<?, ?> context;
private TaskId taskId;

private AtomicInteger numOpenIterators = new AtomicInteger(0);
private LongAdder numOpenIterators = new LongAdder();

@SuppressWarnings("rawtypes")
private final Map<Class, QueryHandler> queryHandlers =
Expand Down Expand Up @@ -137,7 +137,7 @@ private void registerMetrics() {
e2eLatencySensor = StateStoreMetrics.e2ELatencySensor(taskId.toString(), metricsScope, name(), streamsMetrics);
iteratorDurationSensor = StateStoreMetrics.iteratorDurationSensor(taskId.toString(), metricsScope, name(), streamsMetrics);
StateStoreMetrics.addNumOpenIteratorsGauge(taskId.toString(), metricsScope, name(), streamsMetrics,
(config, now) -> numOpenIterators.get());
(config, now) -> numOpenIterators.sum());
}


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -325,7 +325,7 @@ private MeteredTimestampedKeyValueStoreIterator(final KeyValueIterator<Bytes, by
this.valueAndTimestampDeserializer = valueAndTimestampDeserializer;
this.startNs = time.nanoseconds();
this.returnPlainValue = returnPlainValue;
numOpenIterators.incrementAndGet();
numOpenIterators.increment();
}

@Override
Expand Down Expand Up @@ -353,7 +353,7 @@ public void close() {
final long duration = time.nanoseconds() - startNs;
sensor.record(duration);
iteratorDurationSensor.record(duration);
numOpenIterators.decrementAndGet();
numOpenIterators.decrement();
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@

import java.util.Map;
import java.util.Objects;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.LongAdder;

import static org.apache.kafka.common.utils.Utils.mkEntry;
import static org.apache.kafka.common.utils.Utils.mkMap;
Expand All @@ -76,7 +76,7 @@ public class MeteredWindowStore<K, V>
private InternalProcessorContext<?, ?> context;
private TaskId taskId;

private AtomicInteger numOpenIterators = new AtomicInteger(0);
private LongAdder numOpenIterators = new LongAdder();

@SuppressWarnings("rawtypes")
private final Map<Class, QueryHandler> queryHandlers =
Expand Down Expand Up @@ -156,7 +156,7 @@ private void registerMetrics() {
e2eLatencySensor = StateStoreMetrics.e2ELatencySensor(taskId.toString(), metricsScope, name(), streamsMetrics);
iteratorDurationSensor = StateStoreMetrics.iteratorDurationSensor(taskId.toString(), metricsScope, name(), streamsMetrics);
StateStoreMetrics.addNumOpenIteratorsGauge(taskId.toString(), metricsScope, name(), streamsMetrics,
(config, now) -> numOpenIterators.get());
(config, now) -> numOpenIterators.sum());
}

@Deprecated
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@
import org.apache.kafka.streams.StreamsMetrics;
import org.apache.kafka.streams.state.WindowStoreIterator;

import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.LongAdder;
import java.util.function.Function;

class MeteredWindowStoreIterator<V> implements WindowStoreIterator<V> {
Expand All @@ -34,15 +34,15 @@ class MeteredWindowStoreIterator<V> implements WindowStoreIterator<V> {
private final Function<byte[], V> valueFrom;
private final long startNs;
private final Time time;
private final AtomicInteger numOpenIterators;
private final LongAdder numOpenIterators;

MeteredWindowStoreIterator(final WindowStoreIterator<byte[]> iter,
final Sensor operationSensor,
final Sensor iteratorSensor,
final StreamsMetrics metrics,
final Function<byte[], V> valueFrom,
final Time time,
final AtomicInteger numOpenIterators) {
final LongAdder numOpenIterators) {
this.iter = iter;
this.operationSensor = operationSensor;
this.iteratorSensor = iteratorSensor;
Expand All @@ -51,7 +51,7 @@ class MeteredWindowStoreIterator<V> implements WindowStoreIterator<V> {
this.startNs = time.nanoseconds();
this.time = time;
this.numOpenIterators = numOpenIterators;
numOpenIterators.incrementAndGet();
numOpenIterators.increment();
}

@Override
Expand All @@ -73,7 +73,7 @@ public void close() {
final long duration = time.nanoseconds() - startNs;
operationSensor.record(duration);
iteratorSensor.record(duration);
numOpenIterators.decrementAndGet();
numOpenIterators.decrement();
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@
import org.apache.kafka.streams.kstream.Windowed;
import org.apache.kafka.streams.state.KeyValueIterator;

import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.LongAdder;
import java.util.function.Function;

class MeteredWindowedKeyValueIterator<K, V> implements KeyValueIterator<Windowed<K>, V> {
Expand All @@ -37,7 +37,7 @@ class MeteredWindowedKeyValueIterator<K, V> implements KeyValueIterator<Windowed
private final Function<byte[], V> deserializeValue;
private final long startNs;
private final Time time;
private final AtomicInteger numOpenIterators;
private final LongAdder numOpenIterators;

MeteredWindowedKeyValueIterator(final KeyValueIterator<Windowed<Bytes>, byte[]> iter,
final Sensor operationSensor,
Expand All @@ -46,7 +46,7 @@ class MeteredWindowedKeyValueIterator<K, V> implements KeyValueIterator<Windowed
final Function<byte[], K> deserializeKey,
final Function<byte[], V> deserializeValue,
final Time time,
final AtomicInteger numOpenIterators) {
final LongAdder numOpenIterators) {
this.iter = iter;
this.operationSensor = operationSensor;
this.iteratorSensor = iteratorSensor;
Expand All @@ -56,7 +56,7 @@ class MeteredWindowedKeyValueIterator<K, V> implements KeyValueIterator<Windowed
this.startNs = time.nanoseconds();
this.time = time;
this.numOpenIterators = numOpenIterators;
numOpenIterators.incrementAndGet();
numOpenIterators.increment();
}

@Override
Expand All @@ -83,7 +83,7 @@ public void close() {
final long duration = time.nanoseconds() - startNs;
operationSensor.record(duration);
iteratorSensor.record(duration);
numOpenIterators.decrementAndGet();
numOpenIterators.decrement();
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -438,7 +438,7 @@ public static void addNumOpenIteratorsGauge(final String taskId,
final String storeType,
final String storeName,
final StreamsMetricsImpl streamsMetrics,
final Gauge<Integer> numOpenIteratorsGauge) {
final Gauge<Long> numOpenIteratorsGauge) {
streamsMetrics.addStoreLevelMutableMetric(
taskId,
storeType,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -451,13 +451,13 @@ public void shouldTrackOpenIteratorsMetric() {
final KafkaMetric openIteratorsMetric = metric("num-open-iterators");
assertThat(openIteratorsMetric, not(nullValue()));

assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0));
assertThat((Long) openIteratorsMetric.metricValue(), equalTo(0L));

try (final KeyValueIterator<String, String> iterator = metered.prefixScan(KEY, stringSerializer)) {
assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(1));
assertThat((Long) openIteratorsMetric.metricValue(), equalTo(1L));
}

assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0));
assertThat((Long) openIteratorsMetric.metricValue(), equalTo(0L));
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -615,13 +615,13 @@ public void shouldTrackOpenIteratorsMetric() {
final KafkaMetric openIteratorsMetric = metric("num-open-iterators");
assertThat(openIteratorsMetric, not(nullValue()));

assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0));
assertThat((Long) openIteratorsMetric.metricValue(), equalTo(0L));

try (final KeyValueIterator<Windowed<String>, String> iterator = store.backwardFetch(KEY)) {
assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(1));
assertThat((Long) openIteratorsMetric.metricValue(), equalTo(1L));
}

assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0));
assertThat((Long) openIteratorsMetric.metricValue(), equalTo(0L));
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -449,13 +449,13 @@ public void shouldTrackOpenIteratorsMetric() {
final KafkaMetric openIteratorsMetric = metric("num-open-iterators");
assertThat(openIteratorsMetric, not(nullValue()));

assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0));
assertThat((Long) openIteratorsMetric.metricValue(), equalTo(0L));

try (final KeyValueIterator<String, ValueAndTimestamp<String>> iterator = metered.all()) {
assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(1));
assertThat((Long) openIteratorsMetric.metricValue(), equalTo(1L));
}

assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0));
assertThat((Long) openIteratorsMetric.metricValue(), equalTo(0L));
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -380,15 +380,15 @@ public void shouldTrackOpenIteratorsMetric() {
final KafkaMetric openIteratorsMetric = getMetric("num-open-iterators");
assertThat(openIteratorsMetric, not(nullValue()));

assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0));
assertThat((Long) openIteratorsMetric.metricValue(), equalTo(0L));

final QueryResult<VersionedRecordIterator<String>> result = store.query(query, bound, config);

try (final VersionedRecordIterator<String> iterator = result.getResult()) {
assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(1));
assertThat((Long) openIteratorsMetric.metricValue(), equalTo(1L));
}

assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0));
assertThat((Long) openIteratorsMetric.metricValue(), equalTo(0L));
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -456,13 +456,13 @@ public void shouldTrackOpenIteratorsMetric() {
final KafkaMetric openIteratorsMetric = metric("num-open-iterators");
assertThat(openIteratorsMetric, not(nullValue()));

assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0));
assertThat((Long) openIteratorsMetric.metricValue(), equalTo(0L));

try (final KeyValueIterator<Windowed<String>, String> iterator = store.all()) {
assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(1));
assertThat((Long) openIteratorsMetric.metricValue(), equalTo(1L));
}

assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0));
assertThat((Long) openIteratorsMetric.metricValue(), equalTo(0L));
}

@Test
Expand Down