[jira] [Commented] (KAFKA-7591) Changelog retention period doesn't synchronise with window-store size
[ https://issues.apache.org/jira/browse/KAFKA-7591?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17015433#comment-17015433 ] Jon Bates commented on KAFKA-7591: -- Agreed! A WARN message could at least be picked up, even if synchronizing the retention period isn't feasible > Changelog retention period doesn't synchronise with window-store size > - > > Key: KAFKA-7591 > URL: https://issues.apache.org/jira/browse/KAFKA-7591 > Project: Kafka > Issue Type: Improvement > Components: streams >Reporter: Jon Bates >Priority: Major > > When a new windowed state store is created, the associated changelog topic's > `retention.ms` value is set to `window-size + > CHANGELOG_ADDITIONAL_RETENTION_MS` > h3. Expected Behaviour > If the window-size is updated, the changelog topic's `retention.ms` config > should be updated to reflect the new size > h3. Actual Behaviour > The changelog-topic's `retention.ms` setting is not amended, resulting in > possible loss of data upon application restart > > n.b. Although it is easy to update changelog topic config, I logged this as > `major` due to the potential for data-loss for any user of Kafka-Streams who > may not be intimately aware of the relationship between a windowed store and > the changelog config -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Created] (KAFKA-7591) Changelog retention period doesn't synchronise with window-store size
Jon Bates created KAFKA-7591: Summary: Changelog retention period doesn't synchronise with window-store size Key: KAFKA-7591 URL: https://issues.apache.org/jira/browse/KAFKA-7591 Project: Kafka Issue Type: Bug Components: streams Reporter: Jon Bates When a new windowed state store is created, the associated changelog topic's `retention.ms` value is set to `window-size + CHANGELOG_ADDITIONAL_RETENTION_MS` h3. Expected Behaviour If the window-size is updated, the changelog topic's `retention.ms` config should be updated to reflect the new size h3. Actual Behaviour The changelog-topic's `retention.ms` setting is not amended, resulting in possible loss of data upon application restart n.b. Although it is easy to update changelog topic config, I logged this as `major` due to the potential for data-loss for any user of Kafka-Streams who may not be intimately aware of the relationship between a windowed store and the changelog config -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (KAFKA-7192) State-store can desynchronise with changelog
[ https://issues.apache.org/jira/browse/KAFKA-7192?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16555650#comment-16555650 ] Jon Bates commented on KAFKA-7192: -- Great; thanks! We came up with a similar general solution (delete state stores that don't have a checkpoint file), but don't have the insight into Kafka's internals. Just as a sanity check, if the checkpoint file points to a offset behind the changelog tail, does Kafka Streams continue syncing the state store before consuming from the source topics? (I believe this happens already, but its just to aid my understanding)1` > State-store can desynchronise with changelog > > > Key: KAFKA-7192 > URL: https://issues.apache.org/jira/browse/KAFKA-7192 > Project: Kafka > Issue Type: Bug > Components: streams >Affects Versions: 1.1.1 >Reporter: Jon Bates >Priority: Critical > > n.b. this bug has been verified with exactly-once processing enabled > Consider the following scenario: > * A record, N is read into a Kafka topology > * the state store is updated > * the topology crashes > h3. *Expected behaviour:* > # Node is restarted > # Offset was never updated, so record N is reprocessed > # State-store is reset to position N-1 > # Record is reprocessed > h3. *Actual Behaviour* > # Node is restarted > # Record N is reprocessed (good) > # The state store has the state from the previous processing > I'd consider this a corruption of the state-store, hence the critical > Priority, although High may be more appropriate. > I wrote a proof-of-concept here, which demonstrates the problem on Linux: > [https://github.com/spadger/kafka-streams-sad-state-store] -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (KAFKA-7192) State-store can desynchronise with changelog
Jon Bates created KAFKA-7192: Summary: State-store can desynchronise with changelog Key: KAFKA-7192 URL: https://issues.apache.org/jira/browse/KAFKA-7192 Project: Kafka Issue Type: Bug Components: streams Affects Versions: 1.1.1 Reporter: Jon Bates n.b. this bug has been verified with exactly-once processing enabled Consider the following scenario: * A record, N is read into a Kafka topology * the state store is updated * the topology crashes h3. *Expected behaviour:* # Node is restarted # Offset was never updated, so record N is reprocessed # State-store is reset to position N-1 # Record is reprocessed h3. [|https://github.com/spadger/kafka-streams-sad-state-store#actual-behaviour]*Actual Behaviour* # Node is restarted # Record N is reprocessed (good) # The state store has the state from the previous processing I'd consider this a corruption of the state-store, hence the critical Priority, although High may be more appropriate. I wrote a proof-of-concept here, which demonstrates the problem on Linux: https://github.com/spadger/kafka-streams-sad-state-store -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Updated] (KAFKA-7192) State-store can desynchronise with changelog
[ https://issues.apache.org/jira/browse/KAFKA-7192?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jon Bates updated KAFKA-7192: - Description: n.b. this bug has been verified with exactly-once processing enabled Consider the following scenario: * A record, N is read into a Kafka topology * the state store is updated * the topology crashes h3. *Expected behaviour:* # Node is restarted # Offset was never updated, so record N is reprocessed # State-store is reset to position N-1 # Record is reprocessed h3. *Actual Behaviour* # Node is restarted # Record N is reprocessed (good) # The state store has the state from the previous processing I'd consider this a corruption of the state-store, hence the critical Priority, although High may be more appropriate. I wrote a proof-of-concept here, which demonstrates the problem on Linux: [https://github.com/spadger/kafka-streams-sad-state-store] was: n.b. this bug has been verified with exactly-once processing enabled Consider the following scenario: * A record, N is read into a Kafka topology * the state store is updated * the topology crashes h3. *Expected behaviour:* # Node is restarted # Offset was never updated, so record N is reprocessed # State-store is reset to position N-1 # Record is reprocessed h3. [|https://github.com/spadger/kafka-streams-sad-state-store#actual-behaviour]*Actual Behaviour* # Node is restarted # Record N is reprocessed (good) # The state store has the state from the previous processing I'd consider this a corruption of the state-store, hence the critical Priority, although High may be more appropriate. I wrote a proof-of-concept here, which demonstrates the problem on Linux: https://github.com/spadger/kafka-streams-sad-state-store > State-store can desynchronise with changelog > > > Key: KAFKA-7192 > URL: https://issues.apache.org/jira/browse/KAFKA-7192 > Project: Kafka > Issue Type: Bug > Components: streams >Affects Versions: 1.1.1 >Reporter: Jon Bates >Priority: Critical > > n.b. this bug has been verified with exactly-once processing enabled > Consider the following scenario: > * A record, N is read into a Kafka topology > * the state store is updated > * the topology crashes > h3. *Expected behaviour:* > # Node is restarted > # Offset was never updated, so record N is reprocessed > # State-store is reset to position N-1 > # Record is reprocessed > h3. *Actual Behaviour* > # Node is restarted > # Record N is reprocessed (good) > # The state store has the state from the previous processing > I'd consider this a corruption of the state-store, hence the critical > Priority, although High may be more appropriate. > I wrote a proof-of-concept here, which demonstrates the problem on Linux: > [https://github.com/spadger/kafka-streams-sad-state-store] -- This message was sent by Atlassian JIRA (v7.6.3#76005)