KAFKA-6607: Commit correct offsets for transactional input data#8040
KAFKA-6607: Commit correct offsets for transactional input data#8040mjsax merged 6 commits intoapache:2.5from
Conversation
There was a problem hiding this comment.
Some additional side cleanup
There was a problem hiding this comment.
This new method is add for the fix
There was a problem hiding this comment.
To reuse this condition, we move it to out test utils class
There was a problem hiding this comment.
new method create in test utils class to make is reusable
There was a problem hiding this comment.
Need to set retries no a not-zero value for transactions...
There was a problem hiding this comment.
The default value for retries are Integer.MAX_VALUE anyways right?
There was a problem hiding this comment.
Yes, but we set it to zero within TestUtils.producerConfig() (not sure why) -- should we remove it there instead?
There was a problem hiding this comment.
Hmm.. I am not sure either but we can check if removing that breaks any other tests --- with the new deliver timeout value we should no longer rely on that config values, --- if we want, we should change the deliver.timeout not this one.
I guess it's up to you :) if it is too much we can just keep it as is.
There was a problem hiding this comment.
I removed the retries overwrite and left the delivery timeout default. Locally all integration tests passed. Hence, I hope it's fine that way.
There was a problem hiding this comment.
The original method was too long and checkstyle failed -- we could also add a checkstyle exception... This was just a quick fix -- let me know what you think
There was a problem hiding this comment.
Sounds good to me -- better not add more checkstyle exceptions :)
There was a problem hiding this comment.
add verification for the "head record offset"
There was a problem hiding this comment.
Improve some existing tests, and add couple of more that are missing.
There was a problem hiding this comment.
add "head record offset" verification
There was a problem hiding this comment.
Because we call consumer.position() now, we need to fix the mock consumer setup in the TTD
There was a problem hiding this comment.
We cannot share the consumer any longer, because the global task calls unassign() that nukes our setup from above.
There was a problem hiding this comment.
Not clear why? In MockConsumer we only do the following:
public synchronized void unsubscribe() {
ensureNotClosed();
committed.clear();
subscriptions.unsubscribe();
}
And the beginningOffsets map are not nuked.
There was a problem hiding this comment.
The problem is that the subscription is nuked and when we call position() the MockConsumer checks if the passed in partition is in its subscription and fails before it tries to access the beginningOffsets map.
There was a problem hiding this comment.
This was actually detected by the improved tests... Minor side fix.
guozhangwang
left a comment
There was a problem hiding this comment.
Thanks for the PR @mjsax ! The proposed solution looks good to me. Just a few minor comments plus a meta one for consumer.position exception handling.
There was a problem hiding this comment.
We need to consider handling two exceptions that consumer.position may throw: KafkaException -> should be a fatal one; TimeoutException -> in this case we cannot commit, probably have to treat as fatal..
There was a problem hiding this comment.
Ack. We can wrap KafkaException as StreamsException. A TimeoutException should never happen (compare my comment -- let me know if you think the comment is incorrect) -- hence, we can rethrow TimeoutException as IllegalStateException to flag potential bugs.
There was a problem hiding this comment.
The default value for retries are Integer.MAX_VALUE anyways right?
There was a problem hiding this comment.
nit: unkonwn-partition.
There was a problem hiding this comment.
Sounds good to me -- better not add more checkstyle exceptions :)
There was a problem hiding this comment.
Not clear why? In MockConsumer we only do the following:
public synchronized void unsubscribe() {
ensureNotClosed();
committed.clear();
subscriptions.unsubscribe();
}
And the beginningOffsets map are not nuked.
|
Updated. |
|
LGTM! |
|
Java 8: Java 11: Retest this please. |
|
Java 8: Java 11: Retest this please. |
b551ea0 to
fc3cc69
Compare
|
@guozhangwang The Long story short: to fix the issue, I change this PR to get However, with that fix, the Let me know what you think. Also @hachikuji for review of the consumer change. |
There was a problem hiding this comment.
Even if records might not be empty, we need to filter out the dummy records we added to indicate tx-markers
There was a problem hiding this comment.
We add a dummy record is we stepped over an tx-marker
There was a problem hiding this comment.
Instead of nothing, we return an empty list if we step over a tx-marker. After a second fetchRecords() we return nothing.
There was a problem hiding this comment.
We know pass in the current consumerPosition to track it correctly.
There was a problem hiding this comment.
We track the consumer position expliclity now
There was a problem hiding this comment.
This could happen if the buffer is empty and if the consumer only stepped over a commit marker passing in empty list of records.
There was a problem hiding this comment.
Previously, we clear the partitionGroup within closeTopology() that we call above -- however, because of the consumer position tracking, we need to delay it after the commit.
There was a problem hiding this comment.
We add the current consumer position when we add records to the queue now.
There was a problem hiding this comment.
I improve this test a little bit, adding more conditions.
|
@guozhangwang @hachikuji This fix is for KS that does manual commits. I am wondering if consumer |
f4cfcc7 to
1281c0b
Compare
|
@mjsax Hmm.. after reading the PR I'm a bit inclined to fix the "unknown reason" that caused consumer.position to return the wrong value here; I'm still wondering if it is related to consumer only (I read through the code once again just now and cannot find an obvious root cause) or it is related to the interaction of streams / consumer. Can we reproduce this bug with a consumer smoke scenario?
If we believe the consumer.position is buggy then it would affect auto.offset.commit too since we just rely on the position to commit when it triggers. |
|
I basically agree (just wanted to make progress on this PR -- it's easy to revert it back :) -- even if I think that the explicit position tracking is too bad -- not sure about the change in the consumer itself...) -- however, I don't understand the consumer code good enough to fix the consumer (if there is really a bug). Not sure if you or @hachikuji would have time to investigate and verify if there is a bug or if it's a usage issue in Streams? This fix should go into For me, |
1281c0b to
ab3ba68
Compare
ab3ba68 to
7f3c62b
Compare
|
Updated this PR. |
guozhangwang
left a comment
There was a problem hiding this comment.
Reviewed the latest commit, just one minor comment otherwise LGTM!
| // if we are in PENDING_SHUTDOWN and don't find the task it implies that it was a newly assigned | ||
| // task that we just skipped to create; | ||
| // hence, we just skip adding the corresponding records | ||
| continue; |
There was a problem hiding this comment.
maybe we can also log this as INFO for debugging purposes?
|
Java 8 passed. Retest this please. |
|
Java 8: Different test than above. Seems unrelated. Mering this PR. |
|
PR for |
…he#8040) Reviewer: Guozhang Wang <guozhang@confluent.io>
Currently, Kafka Streams commits "offset + 1" that may lead to incorrect "consumer lag" if the input topic is transactional, because the committed offset does "step on" the commit marker, instead of "skipping it".
With this PR, we commit "offsetOfNextRecord" or
consumer.position()to step over potential transactional markers to fix this issue.Call for review @guozhangwang @ableegoldman
This PR is against
2.5branch on purpose to avoid conflict with the current Kafka Streams refactoring. After the refactoring is merged, we can port this PR totrunk.