Skip to content

Conversation

Nikita-Shupletsov
Copy link
Contributor

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

KAFKA-19775: Don't fail if nextOffsetsAndMetadataToBeConsumed is not
available.

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.

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

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