Skip to content

Conversation

@Nikita-Shupletsov
Copy link
Contributor

@Nikita-Shupletsov Nikita-Shupletsov commented Oct 8, 2025

Before we added caching for consumer next offsets we'd called
mainConsumer.position and always expected something back. When we
added the caching, we kept the check that we always have nextOffset, but
as the logic changed to fetching the offsets from poll, we may not have
anything for topics that have no messages. This PR accounts for that.

Reviewers: Lucas Brutschy [email protected], Matthias J. Sax
[email protected]

@github-actions github-actions bot added streams triage PRs from the community labels Oct 8, 2025
}
} catch (final KafkaException fatal) {
throw new StreamsException(fatal);
if (nextOffsetsAndMetadataToBeConsumed.containsKey(partition)) {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

the change that reworked that code to use cached offsets: https://github.com/apache/kafka/pull/17091/files#diff-a76674468cda8772230fb8411717cf9068b1a363a792f32c602fb2ec5ba9efd7R472

@aliehsaeedii @lucasbru, as you folks worked on it, could you please take a look? thanks

break;
return partitionsNeedCommit.stream()
.flatMap(partition -> findOffsetAndMetadata(partition)
.map(offsetAndMetadata -> Map.entry(partition, offsetAndMetadata))
Copy link
Member

Choose a reason for hiding this comment

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

If findOffsetAndMetadata returns an empty Optional, the partition will be dropped, and we does not end up in the computed "committableOffsets" Map, right?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

yes. if findOffsetAndMetadata returns an empty optional, that topic partition will not be in the result list

// This indicates a bug and thus we rethrow it as fatal `IllegalStateException`
throw new IllegalStateException("Stream task " + id + " does not know the partition: " + partition);
}
} catch (final KafkaException fatal) {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

as there are no client calls anymore, the catch is redundant

Copy link
Member

Choose a reason for hiding this comment

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

Nice slide cleanup.

}
break;
return partitionsNeedCommit.stream()
.flatMap(partition -> findOffsetAndMetadata(partition)
Copy link
Member

Choose a reason for hiding this comment

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

Java streams API always confused me. Why do we use flatMap here? Don't we get a single entry in "committableOffsets" Map that we compute here? Or is it needed, because we might drop some partitions?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

so the path here is:
a stream of partitionsNeedCommit -> flatMap(partition -> to offsetAndMetadata optional -> map optional to map.entry optional -> map it to a stream) -> collect

the reason why we map the the optional to a stream is that it looks nicer. otherwise we would need to have two operations: filter(Optonal::ifPresent).map(Optional::get).
if you prefer that syntax, I will update the code

Copy link
Member

Choose a reason for hiding this comment

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

Subjective. :) If would it personally find easy to read using filter(...).map(...); it's more explicit, and less "magic"

} catch (final KafkaException fatal) {
throw new StreamsException(fatal);
if (nextOffsetsAndMetadataToBeConsumed.containsKey(partition)) {
final OffsetAndMetadata offsetAndMetadata = nextOffsetsAndMetadataToBeConsumed.get(partition);
Copy link
Member

Choose a reason for hiding this comment

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

nit: I find the constainsKey() followed by get() pattern always confusing and hard to read... How about

final OffsetAndMetadata offsetAndMetadata = nextOffsetsAndMetadataToBeConsumed.get(partition);
if (offsetAndMetadata != null) {
   offset = offsetAndMetadata.offset();
   leaderEpoch = offsetAndMetadata.leaderEpoch();
}

Just a personal preference.

…ms/integration/RegexSourceIntegrationTest.java

Co-authored-by: Matthias J. Sax <[email protected]>
@mjsax mjsax added ci-approved and removed triage PRs from the community labels Oct 8, 2025
@Nikita-Shupletsov Nikita-Shupletsov changed the title MINOR: Don't fail if nextOffsetsAndMetadataToBeConsumed is not available. KAFKA-19775: Don't fail if nextOffsetsAndMetadataToBeConsumed is not available. Oct 9, 2025
Copy link
Member

@lucasbru lucasbru left a comment

Choose a reason for hiding this comment

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

LGTM, thanks!

@github-actions github-actions bot added the small Small PRs label Oct 10, 2025
final OffsetAndMetadata offsetAndMetadata = nextOffsetsAndMetadataToBeConsumed.get(partition);
if (offsetAndMetadata == null) {
try {
offset = mainConsumer.position(partition);
Copy link
Contributor Author

Choose a reason for hiding this comment

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

my understanding so far:
we have some metadata that's one per task. but we have multiple input partitions. so instead of picking one, we just add it everywhere. and during restore we just read them all in whatever order and expect them to be more or less the same.
so if we are adding a new partition on the fly, we want that partition to also have that metadata. if we follow that logic.
so if we follow that logic, we need to commit even for that empty partition.

so I added a fallback to the previous logic when we ask the consumer about the offset

Copy link
Member

Choose a reason for hiding this comment

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

Yes, this is also what I understood when digging into it last week. I'm actually not sure that logic is fully watertight - what if my regular expression matches a disjoint subset of topics - none of them will have the metadata, so I have lost it, right? It seems a bit of a best-effort thing to me... In which we could also consider not committing the "partition we never consumed from" here.

} catch (final TimeoutException error) {
// the `consumer.position()` call should never block, because we know that we did process data
// for the requested partition and thus the consumer should have a valid local position
// that it can return immediately
Copy link
Member

Choose a reason for hiding this comment

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

Is that still valid in the corner case? I think in the case you are describing, we haven't processed any records so we may not have a position for the partition yet and this will actually block to do an offset fetch to find the last committed offset for the partition?

@github-actions github-actions bot removed the small Small PRs label Oct 15, 2025
@mjsax mjsax merged commit 44481ca into apache:trunk Oct 16, 2025
30 of 35 checks passed
mjsax pushed a commit that referenced this pull request Oct 16, 2025
…available. (#20665)

Before we added caching for consumer next offsets we'd called
`mainConsumer.position` and always expected something back. When we
added the caching, we kept the check that we always have nextOffset, but
as the logic changed to fetching the offsets from poll, we may not have
anything for topics that have no messages. This PR accounts for that.

Reviewers: Lucas Brutschy <[email protected]>, Matthias J. Sax
 <[email protected]>
mjsax pushed a commit that referenced this pull request Oct 16, 2025
…available. (#20665)

Before we added caching for consumer next offsets we'd called
`mainConsumer.position` and always expected something back. When we
added the caching, we kept the check that we always have nextOffset, but
as the logic changed to fetching the offsets from poll, we may not have
anything for topics that have no messages. This PR accounts for that.

Reviewers: Lucas Brutschy <[email protected]>, Matthias J. Sax
 <[email protected]>
@mjsax
Copy link
Member

mjsax commented Oct 16, 2025

Thanks for the fix! -- Merged to trunk and cherry-picked to 4.1 and 4.0 branches.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants