[ https://issues.apache.org/jira/browse/KAFKA-4322?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15657841#comment-15657841 ]
Guozhang Wang commented on KAFKA-4322: -------------------------------------- [~markshelton] Thanks for proposing this ticket. I read the PR and here are some meta-questions: 1. Currently we do not commit offset for the restored consumer since we store the restored offset in a separate checkpoint offset file, which indicate both from where the restoration should be starting by fetching from the changelog topic, but also as a flag hinting that the previous run of the instance was shutdown cleanly, and hence the local storage engine files (e.g. RocksDB folders) can be re-used. Is there any particular reasons that you want to commit offsets after restoration? 2. We are improving our metrics / logging mechanism as a major effort for better operation experience as well as debuggability. This will also include the number of records restored. So beyond this usage, I'm wondering if you have other common requests that would benefit from the additional callbacks? If you feel that this is still a common feature that we should add to Kafka, could you add a KIP proposal (https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals) since it contains changes to the public APIs for Kafka clients, and we can continue our discussion there. BTW I have also added you to the contributor list of Apache Kafka, from now on you should be able to assign JIRAs to yourself. > StateRestoreCallback begin and end indication > --------------------------------------------- > > Key: KAFKA-4322 > URL: https://issues.apache.org/jira/browse/KAFKA-4322 > Project: Kafka > Issue Type: Improvement > Components: streams > Affects Versions: 0.10.0.1 > Reporter: Mark Shelton > Assignee: Mark Shelton > Priority: Minor > > In Kafka Streams, the StateRestoreCallback interface provides only a single > method "restore(byte[] key, byte[] value)" that is called for every key-value > pair to be restored. > It would be nice to have "beginRestore" and "endRestore" methods as part of > StateRestoreCallback. > Kafka Streams would call "beginRestore" before restoring any keys, and would > call "endRestore" when it determines that it is done. This allows an > implementation, for example, to report on the number of keys restored and > perform a commit after the last key was restored. Other uses are conceivable. -- This message was sent by Atlassian JIRA (v6.3.4#6332)