Nikita-Shupletsov commented on code in PR #20665:
URL: https://github.com/apache/kafka/pull/20665#discussion_r2415233643
##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java:
##########
@@ -465,38 +465,30 @@ public Map<TopicPartition, OffsetAndMetadata>
prepareCommit(final boolean clean)
}
}
- private OffsetAndMetadata findOffsetAndMetadata(final TopicPartition
partition) {
+ private Optional<OffsetAndMetadata> findOffsetAndMetadata(final
TopicPartition partition) {
Long offset = partitionGroup.headRecordOffset(partition);
Optional<Integer> leaderEpoch =
partitionGroup.headRecordLeaderEpoch(partition);
final long partitionTime =
partitionGroup.partitionTimestamp(partition);
if (offset == null) {
- try {
- if (nextOffsetsAndMetadataToBeConsumed.containsKey(partition))
{
- final OffsetAndMetadata offsetAndMetadata =
nextOffsetsAndMetadataToBeConsumed.get(partition);
- offset = offsetAndMetadata.offset();
- leaderEpoch = offsetAndMetadata.leaderEpoch();
- } else {
- // 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) {
Review Comment:
as there are no client calls anymore, the catch is redundant
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]