Nikita-Shupletsov commented on code in PR #20665:
URL: https://github.com/apache/kafka/pull/20665#discussion_r2415240265


##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java:
##########
@@ -505,12 +497,12 @@ private Map<TopicPartition, OffsetAndMetadata> 
committableOffsetsAndMetadata() {
                 // input partitions
                 final Set<TopicPartition> partitionsNeedCommit = 
processorContext.processorMetadata().needsCommit() ?
                     inputPartitions() : consumedOffsets.keySet();
-                committableOffsets = new 
HashMap<>(partitionsNeedCommit.size());
 
-                for (final TopicPartition partition : partitionsNeedCommit) {
-                    committableOffsets.put(partition, 
findOffsetAndMetadata(partition));
-                }
-                break;
+                return partitionsNeedCommit.stream()
+                        .flatMap(partition -> findOffsetAndMetadata(partition)

Review Comment:
   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



-- 
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]

Reply via email to