Skip to content

KAFKA-19407 Fix potential IllegalStateException when appending to timeIndex #19972

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
Jun 24, 2025

Conversation

ocadaruma
Copy link
Contributor

@ocadaruma ocadaruma commented Jun 16, 2025

Summary

Reviewers: Vincent PÉRICART [email protected], Jun Rao
[email protected], Chia-Ping Tsai [email protected]

@github-actions github-actions bot added triage PRs from the community storage Pull requests that target the storage module small Small PRs labels Jun 16, 2025
@ijuma ijuma requested a review from junrao June 16, 2025 23:31
@ocadaruma
Copy link
Contributor Author

ocadaruma commented Jun 17, 2025

This PR is supposed to introduce no any new lock contention, because (potentially blocking) lazy timeIndex materialization has already exclusive control by lock in LazyIndex#get

@@ -192,8 +192,13 @@ public void sanityCheck(boolean timeIndexFileNewlyCreated) throws IOException {
* the time index).
*/
public TimestampOffset readMaxTimestampAndOffsetSoFar() throws IOException {
if (maxTimestampAndOffsetSoFar == TimestampOffset.UNKNOWN)
maxTimestampAndOffsetSoFar = timeIndex().lastEntry();
if (maxTimestampAndOffsetSoFar == TimestampOffset.UNKNOWN) {
Copy link

Choose a reason for hiding this comment

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

Should this volatile field become an Atomic instead?

Copy link
Contributor Author

@ocadaruma ocadaruma Jun 17, 2025

Choose a reason for hiding this comment

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

Using Atomic doesn't make the thing simple, because we need double-check locking anyways to ensure:

  • initialize the value only once at first time
  • always return initialized value

Copy link

Choose a reason for hiding this comment

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

I thought updateAndGet could work to this effect

maxTimestampAndOffsetSoFar.updateAndGet(t -> if (t == TimestampOffset.UNKNOWN) timeIndex().lastEntry() else t)

But I have not fully considered whether it would end up being slower on any microbenchmark. It just looked simpler.

Copy link
Member

@chia7712 chia7712 left a comment

Choose a reason for hiding this comment

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

@ocadaruma nice find!

@github-actions github-actions bot removed the triage PRs from the community label Jun 18, 2025
if (maxTimestampAndOffsetSoFar == TimestampOffset.UNKNOWN)
maxTimestampAndOffsetSoFar = timeIndex().lastEntry();
if (maxTimestampAndOffsetSoFar == TimestampOffset.UNKNOWN) {
synchronized (this) {
Copy link
Member

Choose a reason for hiding this comment

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

The monitor lock of LogSegment may be used by external code. Perhaps it would be safer to use the monitor lock of an internal Object instead?

private final Object maxTimestampAndOffsetLock = new Object();

synchronized (maxTimestampAndOffsetLock) {
}

Copy link
Contributor Author

Choose a reason for hiding this comment

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

fair enough. fixed.

when(seg.timeIndex()).thenReturn(mockTimeIndex);
List<Future<?>> futures = new ArrayList<>();
for (int i = 0; i < numThreads; i++) {
futures.add(executor.submit(() -> {
Copy link
Member

Choose a reason for hiding this comment

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

futures.add(executor.submit(() -> assertDoesNotThrow(seg::maxTimestampSoFar)));

Copy link
Member

Choose a reason for hiding this comment

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

or we can use CompletableFuture to streamline the code.

        long remainingDurationNanos = Duration.ofSeconds(1).toNanos();
        while (remainingDurationNanos > 0) {
            long t0 = System.nanoTime();
            clearInvocations(mockTimeIndex);
            try (LogSegment seg = spy(LogTestUtils.createSegment(0, logDir, 10, Time.SYSTEM))) {
                when(seg.timeIndex()).thenReturn(mockTimeIndex);
                var futures = IntStream.range(0, numThreads).mapToObj(ignored -> CompletableFuture.runAsync(() -> assertDoesNotThrow(seg::maxTimestampSoFar))).toList();
                futures.forEach(CompletableFuture::join);
                // timeIndex.lastEntry should be called once if no race
                verify(mockTimeIndex, times(1)).lastEntry();

                long elapsedNanos = System.nanoTime() - t0;
                remainingDurationNanos -= elapsedNanos;
            }
        }

Copy link
Contributor Author

Choose a reason for hiding this comment

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

thanks, fixed to use assertDoesNotThrow

@ocadaruma ocadaruma requested a review from chia7712 June 22, 2025 20:31
if (maxTimestampAndOffsetSoFar == TimestampOffset.UNKNOWN)
maxTimestampAndOffsetSoFar = timeIndex().lastEntry();
if (maxTimestampAndOffsetSoFar == TimestampOffset.UNKNOWN) {
synchronized (maxTimestampAndOffsetLock) {
Copy link
Member

Choose a reason for hiding this comment

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

Could you please add comments for this lock? By default, LogSegment is not thread-safe so all modification should be executed within UnifiedLog lock. However, there is an exceptional path that could change the maxTimestampAndOffsetSoFar without UnifiedLog lock, which causes this concurrent issue ...

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Sounds good. Added a comment

Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@ocadaruma : Thanks for the PR. LGTM

@chia7712 chia7712 merged commit 959021d into apache:trunk Jun 24, 2025
26 checks passed
chia7712 pushed a commit that referenced this pull request Jun 24, 2025
…eIndex (#19972)

## Summary
- Fix potential race condition in
LogSegment#readMaxTimestampAndOffsetSoFar(), which may result in
non-monotonic offsets and causes replication to stop.
- See https://issues.apache.org/jira/browse/KAFKA-19407 for the details
how it happen.

Reviewers: Vincent PÉRICART <[email protected]>, Jun Rao
 <[email protected]>, Chia-Ping Tsai <[email protected]>
chia7712 pushed a commit that referenced this pull request Jun 24, 2025
…eIndex (#19972)

## Summary
- Fix potential race condition in
LogSegment#readMaxTimestampAndOffsetSoFar(), which may result in
non-monotonic offsets and causes replication to stop.
- See https://issues.apache.org/jira/browse/KAFKA-19407 for the details
how it happen.

Reviewers: Vincent PÉRICART <[email protected]>, Jun Rao
 <[email protected]>, Chia-Ping Tsai <[email protected]>
jiafu1115 pushed a commit to jiafu1115/kafka that referenced this pull request Jul 2, 2025
…eIndex (apache#19972)

## Summary
- Fix potential race condition in
LogSegment#readMaxTimestampAndOffsetSoFar(), which may result in
non-monotonic offsets and causes replication to stop.
- See https://issues.apache.org/jira/browse/KAFKA-19407 for the details
how it happen.

Reviewers: Vincent PÉRICART <[email protected]>, Jun Rao
 <[email protected]>, Chia-Ping Tsai <[email protected]>
jiafu1115 pushed a commit to jiafu1115/kafka that referenced this pull request Jul 3, 2025
…eIndex (apache#19972)

## Summary
- Fix potential race condition in
LogSegment#readMaxTimestampAndOffsetSoFar(), which may result in
non-monotonic offsets and causes replication to stop.
- See https://issues.apache.org/jira/browse/KAFKA-19407 for the details
how it happen.

Reviewers: Vincent PÉRICART <[email protected]>, Jun Rao
 <[email protected]>, Chia-Ping Tsai <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
ci-approved small Small PRs storage Pull requests that target the storage module
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants