Github user revans2 commented on a diff in the pull request:
https://github.com/apache/storm/pull/1679#discussion_r87009044
--- Diff:
external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java
---
@@ -479,16 +487,17 @@ public OffsetAndMetadata findNextCommitOffset() {
KafkaSpoutMessageId nextCommitMsg = null; // this is a
convenience variable to make it faster to create OffsetAndMetadata
for (KafkaSpoutMessageId currAckedMsg : ackedMsgs) { //
complexity is that of a linear scan on a TreeMap
- if ((currOffset = currAckedMsg.offset()) ==
initialFetchOffset || currOffset == nextCommitOffset + 1) { // found
the next offset to commit
+ if ((currOffset = currAckedMsg.offset()) ==
nextCommitOffset + 1) { // found the next offset to commit
found = true;
nextCommitMsg = currAckedMsg;
nextCommitOffset = currOffset;
} else if (currAckedMsg.offset() > nextCommitOffset + 1) {
// offset found is not continuous to the offsets listed to go in the next
commit, so stop search
LOG.debug("topic-partition [{}] has non-continuous
offset [{}]. It will be processed in a subsequent batch.", tp, currOffset);
break;
} else {
- LOG.debug("topic-partition [{}] has unexpected offset
[{}].", tp, currOffset);
- break;
+ //Received a redundant ack. Ignore and continue
processing.
--- End diff --
The issue is not with storm itself, but with the internal bookkeeping of
the spout. If Kafka rebalances partitions (or there are other recoverable
errors) the spout resets all of it's internal bookkeeping, but there is no way
to un-emit something. So everything that was emitted, but not fully
acked/committed when the recovery happened is still outstanding, and will come
back for the spout to process.
1. emit A offset B
2. oh crap, recovering...
3. Recovered handling partition A again
4. emit A offset B again
5. ack A offset B (the first one)
6. ack A offset B (the second recovered one)
---
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 [email protected] or file a JIRA ticket
with INFRA.
---