[ https://issues.apache.org/jira/browse/KAFKA-10493?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17329373#comment-17329373 ]
Matthias J. Sax commented on KAFKA-10493: ----------------------------------------- {quote}verify the order during restoration {quote} I don't think this would work. If we have input `<k,a,10> <k,b,5>` and process the records first, we would discard the second record. However, after log compaction run, the topic would only contain the second record `<k,b,5>` and thus we don't recognize it as an out-of-order record any longer. {quote}disable the source topic optimization entirely {quote} Sounds like a step backwards? {quote}live with the inconsistency and warn users when they enable the source topic optimization {quote} I would rather not do this, because we should make it simpler to users, not harder. {quote}let users enable/disable dropping out-of-order updates per table {quote} If we believe that dropping out-of-order message is the right thing to do semantically, it seems not a good option to let users enable "incorrect" behavior? IMHO it's ok if the DSL is opinionated – if users need different semantics, they can fall back to the processor API. Another "trick" they could apply is, to use a custom timestamp extractor, and `return max(r.tx, partitionTime)` to modify timestamps. {quote}let users enable/disable source topic optimizaton per table (that has been already under discussion if I remember correctly) {quote} On a per table basis maybe. And it's even possible to today to avoid it by rewriting the program from `builder.table()` to `builder.stream().toTable()` – But similarly to the point above, it put a lot of burden on the user to understand the details. {quote}By default, I would enable dropping out-of-order updates and disabling the source topic optimization. {quote} Topology optimization is already disabled by default. > 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)