[ https://issues.apache.org/jira/browse/KAFKA-9909?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17100186#comment-17100186 ]
Guozhang Wang commented on KAFKA-9909: -------------------------------------- I think the reason here is that you catch the exception and continue, i.e. let's say you have three records {{a,b,c}} with offset {{1,2,3}} where {{b}} is ill-formatted: 1. {{a}} is processed normally, and then committed. 2. {{b}} found ill-formatted, and then skipped. 3. {{c}} is still processed normally, and then committed. Now we would commit up to {{3}} which would include {{b}}'s offset {{2}}. Even in consumer, we do not support things like "offset 1 and 3 are committed, but offset 2 is skipped". I.e. if you do not want to commit offset {{2}}, you'd have to either send the record to a queue to bookkeep it, or just stop the app immediately and do not continue to process and commit {{c}}. You can read this section https://docs.confluent.io/current/streams/faq.html#failure-and-exception-handling for some more suggestions. > Kafka Streams : offset control to Streams API > --------------------------------------------- > > Key: KAFKA-9909 > URL: https://issues.apache.org/jira/browse/KAFKA-9909 > Project: Kafka > Issue Type: Improvement > Components: streams > Affects Versions: 2.5.0 > Environment: All > Reporter: Gopikrishna > Priority: Minor > Labels: Offset, commit > > Hello team, really inspired the way streams api is running today. I would > like to have a feature to be flexible regarding the offset. when we write the > processor api, processor context object can be used to commit the offset. > this is not effective. but streams are controlling the offset. the moment the > process method executed or scheduled window completed, the offset is > committed automatically by streams internally. > Like traditional kafka consumer, its better the context object should have > complete control over the offset whether to commit or not. This will give > more control to the api to handle failovers and especially when message > cannot be processed, context should not commit the offset. Appreciate this > can be implemented. > > h4. enable.auto.commit is by default false, but streams are committing > automatically the offset. -- This message was sent by Atlassian Jira (v8.3.4#803005)