GitHub user vivekmittal opened a pull request: https://github.com/apache/storm/pull/2104
[STORM-2505] Spout to support topic compaction Kafka maintains the spout progress (offsets for partitions) which can hold a value which no longer exists (or offset+1 doesn't exist) in the topic due to following reasons Topology stopped processing (or died) & topic got compacted (cleanup.policy=compact) leaving offset voids in the topic. Topology stopped processing (or died) & Topic got cleaned up (cleanup.policy=delete) and the offset. When the topology starts processing again (or restarted), the spout logic suggests that the next offset has to be (committedOffset+1) for the spout to make progress, which will never be the case as (committedOffset+1) has been removed from the topic and will never be acked. **OffsetManager.java** ` if (currOffset == nextCommitOffset + 1) { // found the next offset to commit found = true; nextCommitMsg = currAckedMsg; nextCommitOffset = currOffset; } else if (currOffset > nextCommitOffset + 1) { LOG.debug("topic-partition [{}] has non-continuous offset [{}]. It will be processed in a subsequent batch.", tp, currOffset); } ` This pull request maintains a emitted offset set in OffsetManager & smartly forward the offset to the next logical offset You can merge this pull request into a Git repository by running: $ git pull https://github.com/vivekmittal/storm master Alternatively you can review and apply these changes as the patch at: https://github.com/apache/storm/pull/2104.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #2104 ---- commit acfd13b9c71f6cf4d7307aa1598e8b50b06b0f37 Author: Vivek Mittal <vivek.mit...@flipkart.com> Date: 2017-05-08T05:13:36Z [STORM-2505] Spout to support topic compaction. Maintaining a emitted set in OffsetManager to handle the voids in the topic ---- --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---