[ https://issues.apache.org/jira/browse/KAFKA-8828?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16917947#comment-16917947 ]
Adam Rinehart commented on KAFKA-8828: -------------------------------------- I'm saying that when I use StreamBuilder.globalKTable(), it internally calls addGlobalStore with a StoreBuilder<TimestampedKeyValueStore<K, V>>. We are not explicitly calling addGlobalStore; this is implicitly done in the TableSourceNode.writeToTopology method for us and the StoreBuilder type it uses was changed in 2.3. In the Transformer.init() method, when we call ProcessorContext.getStateStore, it is now returning the TimestampedKeyValueStore that was implicitly added. The part of the code is doing what you expected. The issue is this internal change from KeyValueStore to TimestampedKeyValueStore in TableSourceNode.writeToTopology is exposed through the ProcessorContext.getStateStore, and that exposure is not covered in the upgrade notes. If I get the globalStore via the KafkaStreams.store() method, I have the option to specify the interface I'm getting back, via the queryableStoreType. If I get the globalStore via the ProcessorContext.getStoreState, the interface I need to cast to changed and I don't have the option of specifying it. > [BC Break] Global store returns a TimestampedKeyValueStore in 2.3 > ----------------------------------------------------------------- > > Key: KAFKA-8828 > URL: https://issues.apache.org/jira/browse/KAFKA-8828 > Project: Kafka > Issue Type: Bug > Components: streams > Affects Versions: 2.3.0 > Reporter: Marcos Passos > Priority: Major > > Since 2.3, {{ProcessorContext}} returns a {{TimestampedKeyValueStore}} for > global stores, which is backward incompatible. This change makes the upgrade > path a lot painful and involves creating a non-trivial adapter to hide the > timestamp-related functionality in cases where it is not needed. -- This message was sent by Atlassian Jira (v8.3.2#803003)