bbejeck commented on code in PR #17686:
URL: https://github.com/apache/kafka/pull/17686#discussion_r1830256644


##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java:
##########
@@ -1073,10 +1073,13 @@ private void 
initializeTaskTimeAndProcessorMetadata(final Map<TopicPartition, Of
     @Override
     public Map<TopicPartition, Long> purgeableOffsets() {
         final Map<TopicPartition, Long> purgeableConsumedOffsets = new 
HashMap<>();
-        for (final Map.Entry<TopicPartition, Long> entry : 
consumedOffsets.entrySet()) {
+        for (final Map.Entry<TopicPartition, Long> entry : 
committedOffsets.entrySet()) {
             final TopicPartition tp = entry.getKey();
             if (topology.isRepartitionTopic(tp.topic())) {
-                purgeableConsumedOffsets.put(tp, entry.getValue() + 1);
+                // committedOffsets map is initialized at -1 so no purging 
until there's a committed offset
+                if (entry.getValue() > -1) {
+                    purgeableConsumedOffsets.put(tp, entry.getValue() + 1);

Review Comment:
   We don't explicitly do a `+1` but we get the entry for `commitedOffset` from 
`ConsumerRecord.nextOffset()` so we don't need to add it here anymore.



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