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

Open
wants to merge 1 commit into
base: trunk
Choose a base branch
from

Conversation

ocadaruma
Copy link
Contributor

Summary

@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!

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

Choose a reason for hiding this comment

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

Perhaps we can prevent readMaxTimestampAndOffsetSoFar from changing maxTimestampAndOffsetSoFar by initializing the latter with timeIndex().lastEntry() in the constructor?

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.

If we do so, memory map for all segments will be created eagerly at startup and make it slow against the motivation of lazy index loading

Copy link
Member

Choose a reason for hiding this comment

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

memory map for all segments will be created eagerly at startup

You're right.

The lock approach works for me, but maybe there's another approach that doesn't need a lock. We can pre-load the maxTimestampAndOffsetSoFar at startup for the active segment. If I understand this issue correctly, the issue happens only in the active segment. Fortunately, the memory map of the active segment is always pre-loaded. Hence, we can initialize the maxTimestampAndOffsetSoFar at the same time. WDYT?

@github-actions github-actions bot removed the triage PRs from the community label Jun 18, 2025
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.

4 participants