I expect the join to
execute after the 25 with one side of the join containing a record and the
other being null
Given that you also have a grace period of 5 minutes, the result will
only be emitted after the grace-period passed and the window is closed
(not when window end time is reached).
One has a
naming convention of "KSTREAM_OUTERSHARED". I see a record there, but I'm
not sure how to decode that message to see what is in it. What is the
purpose of those messages?
It's an internal store, that stores all records which are subject to be
emitted as left/right join result, ie, if there is no inner join result.
The format used is internal:
https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/state/internals/LeftOrRightValueSerde.java
Also note: time is based on event-time, ie, if the input stream stops to
send new records, "stream-time" will stop to advance and the result
might not be emitted because the window does not get closed.
(Last, there is some internal wall-clock time delay of one second to
emit results for performance reasons...)
HTH.
-Matthias
On 4/30/24 6:51 AM, Chad Preisler wrote:
Hello,
I have a KStream to KStream outer join with a time difference of 25 minutes
and 5 minutes of grace. When I get a record for one side of the join, but
don't get a record on the other side of the join, I expect the join to
execute after the 25 with one side of the join containing a record and the
other being null. Is that correct? If it is correct, it's not working for
me.
I was poking around on the broker and saw some internal topics. I see the
key I expected to execute the join on some of those topics. One has a
naming convention of "KSTREAM_OUTERSHARED". I see a record there, but I'm
not sure how to decode that message to see what is in it. What is the
purpose of those messages? If I decode the message will it help me see when
the join should have been executed?
I also see the key on a topic with the naming convention
"KSTREAM_OUTERTHIS".
Are there any other topics that I should be looking at to troubleshoot this
issue?
Thanks,
Chad