A. Sophie Blee-Goldman created KAFKA-14976:
----------------------------------------------

             Summary: Left/outer stream-stream joins create KV stores that 
aren't customizable
                 Key: KAFKA-14976
                 URL: https://issues.apache.org/jira/browse/KAFKA-14976
             Project: Kafka
          Issue Type: Bug
          Components: streams
            Reporter: A. Sophie Blee-Goldman


It appears that we only give the illusion of full customizability when it comes 
to the state stores of a windowed join. This arose due to an 
[optimization|https://github.com/apache/kafka/pull/11252] for the performance 
of the spurious results fix, and means that these joins now come with one 
additional, and possibly unexpected, state store:

 
{code:java}
final StoreBuilder<KeyValueStore<TimestampedKeyAndJoinSide<K>, 
LeftOrRightValue<V1, V2>>> builder =
            new ListValueStoreBuilder<>(
         |--[   persistent ? 
this-->  |         Stores.persistentKeyValueStore(storeName) : 
         |--[      Stores.inMemoryKeyValueStore(storeName),
                timestampedKeyAndJoinSideSerde,
                leftOrRightValueSerde,
                Time.SYSTEM
            ); {code}
 

where persistent is defined above that as
{code:java}
final boolean persistent = streamJoinedInternal.thisStoreSupplier() == null || 
streamJoinedInternal.thisStoreSupplier().get().persistent(); {code}
 
This means regardless of whether a custom state store implementation was passed 
in to the join, we will still insert one of our RocksDB or InMemory state 
stores. Which might be very surprising since the API makes it seem like the 
underlying stores are fully configurable.

I'm adding a warning line for this in PR 
[#13682|https://github.com/apache/kafka/pull/13682/files#diff-9ce43046fdef1233ab762e728abd1d3d44d7c270b28dcf6b63aa31a93a30af07R334-R336]
 but we should really make this hidden state store fully configurable like the 
window stores currently are (which will require a KIP)

 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to