Skip to content

MINOR: Fix Streams Position thread-safety #19480

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

Merged
merged 3 commits into from
Apr 16, 2025
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 @@ -104,15 +104,11 @@ public Position merge(final Position other) {
} else {
for (final Entry<String, ConcurrentHashMap<Integer, Long>> entry : other.position.entrySet()) {
final String topic = entry.getKey();
final Map<Integer, Long> partitionMap =
position.computeIfAbsent(topic, k -> new ConcurrentHashMap<>());

for (final Entry<Integer, Long> partitionOffset : entry.getValue().entrySet()) {
final Integer partition = partitionOffset.getKey();
final Long offset = partitionOffset.getValue();
if (!partitionMap.containsKey(partition)
|| partitionMap.get(partition) < offset) {
partitionMap.put(partition, offset);
}
withComponent(topic, partition, offset);
}
}
return this;
Expand Down
127 changes: 127 additions & 0 deletions streams/src/test/java/org/apache/kafka/streams/query/PositionTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,21 @@

import org.junit.jupiter.api.Test;

import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;

import static org.apache.kafka.common.utils.Utils.mkEntry;
import static org.apache.kafka.common.utils.Utils.mkMap;
Expand All @@ -32,9 +42,12 @@
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;

public class PositionTest {

private static final Random RANDOM = new Random();

@Test
public void shouldCreateFromMap() {
final Map<String, Map<Integer, Long>> map = mkMap(
Expand Down Expand Up @@ -221,4 +234,118 @@ public void shouldNotHash() {
final HashMap<Position, Integer> map = new HashMap<>();
assertThrows(UnsupportedOperationException.class, () -> map.put(position, 5));
}

@Test
public void shouldMonotonicallyIncreasePartitionPosition() throws InterruptedException, ExecutionException, TimeoutException {
final int threadCount = 10;
final int maxTopics = 50;
final int maxPartitions = 50;
final int maxOffset = 1000;
final CountDownLatch startLatch = new CountDownLatch(threadCount);
final Position mergePosition = Position.emptyPosition();
final Position withComponentPosition = Position.emptyPosition();
final List<Future<?>> futures = new ArrayList<>();
ExecutorService executorService = null;

try {
executorService = Executors.newFixedThreadPool(threadCount);

for (int i = 0; i < threadCount; i++) {
futures.add(executorService.submit(() -> {
final Position threadPosition = Position.emptyPosition();
final int topicCount = RANDOM.nextInt(maxTopics) + 1;

// build the thread's position
for (int topicNum = 0; topicNum < topicCount; topicNum++) {
final String topic = "topic-" + topicNum;
final int partitionCount = RANDOM.nextInt(maxPartitions) + 1;
for (int partitionNum = 0; partitionNum < partitionCount; partitionNum++) {
final long offset = RANDOM.nextInt(maxOffset) + 1;
threadPosition.withComponent(topic, partitionNum, offset);
}
}

startLatch.countDown();
try {
startLatch.await();
} catch (final InterruptedException e) {
// convert to unchecked exception so the future completes exceptionally and fails the test
throw new RuntimeException(e);
}

// merge with the shared position
mergePosition.merge(threadPosition);
// duplicate the shared position to get a snapshot of its state
final Position threadMergePositionState = mergePosition.copy();

// update the shared position using withComponent
for (final String topic : threadPosition.getTopics()) {
for (final Map.Entry<Integer, Long> partitionOffset : threadPosition
.getPartitionPositions(topic)
.entrySet()) {
withComponentPosition.withComponent(topic, partitionOffset.getKey(), partitionOffset.getValue());
}
}
// duplicate the shared position to get a snapshot of its state
final Position threadWithComponentPositionState = withComponentPosition.copy();

// validate that any offsets in the merged position and the withComponent position are >= the thread position
for (final String topic : threadPosition.getTopics()) {
final Map<Integer, Long> threadOffsets = threadPosition.getPartitionPositions(topic);
final Map<Integer, Long> mergedOffsets = threadMergePositionState.getPartitionPositions(topic);
final Map<Integer, Long> withComponentOffsets = threadWithComponentPositionState.getPartitionPositions(topic);

for (final Map.Entry<Integer, Long> threadOffset : threadOffsets.entrySet()) {
final int partition = threadOffset.getKey();
final long offsetValue = threadOffset.getValue();

// merge checks
assertTrue(
mergedOffsets.containsKey(partition),
"merge method failure. Missing partition " + partition + " for topic " + topic
);
assertTrue(
mergedOffsets.get(partition) >= offsetValue,
"merge method failure. Offset for topic " +
topic +
" partition " +
partition +
" expected >= " +
offsetValue +
" but got " +
mergedOffsets.get(partition)
);

// withComponent checks
assertTrue(
withComponentOffsets.containsKey(partition),
"withComponent method failure. Missing partition " + partition + " for topic " + topic
);
assertTrue(
withComponentOffsets.get(partition) >= offsetValue,
"withComponent method failure. Offset for topic " +
topic +
" partition " +
partition +
" expected >= " +
offsetValue +
" but got " +
withComponentOffsets.get(partition)
);
}
}
}));
}

for (final Future<?> future : futures) {
// Wait for all threads to complete
future.get(1, TimeUnit.SECONDS); // Check for exceptions
}
} finally {
if (executorService != null) {
executorService.shutdown();
assertTrue(executorService.awaitTermination(10, TimeUnit.SECONDS));
}
}
}
}