[ https://issues.apache.org/jira/browse/KAFKA-10493?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17327212#comment-17327212 ]
Bruno Cadonna commented on KAFKA-10493: --------------------------------------- The options that I see are: # verify the order during restoration # disable the source topic optimization entirely # live with the inconsistency and warn users when they enable the source topic optimization # let users enable/disable dropping out-of-order updates per table # let users enable/disable source topic optimizaton per table (that has been already under discussion if I remember correctly) I think a mix of 3, 4, and 5 would be the most flexible solution. By default, I would enable dropping out-of-order updates and disabling the source topic optimization. > KTable out-of-order updates are not being ignored > ------------------------------------------------- > > Key: KAFKA-10493 > URL: https://issues.apache.org/jira/browse/KAFKA-10493 > Project: Kafka > Issue Type: Bug > Components: streams > Affects Versions: 2.6.0 > Reporter: Pedro Gontijo > Assignee: Matthias J. Sax > Priority: Blocker > Fix For: 3.0.0 > > Attachments: KTableOutOfOrderBug.java > > > On a materialized KTable, out-of-order records for a given key (records which > timestamp are older than the current value in store) are not being ignored > but used to update the local store value and also being forwarded. > I believe the bug is here: > [https://github.com/apache/kafka/blob/2.6.0/streams/src/main/java/org/apache/kafka/streams/state/internals/ValueAndTimestampSerializer.java#L77] > It should return true, not false (see javadoc) > The bug impacts here: > [https://github.com/apache/kafka/blob/2.6.0/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java#L142-L148] > I have attached a simple stream app that shows the issue happening. > Thank you! -- This message was sent by Atlassian Jira (v8.3.4#803005)