Hi Matthias,
Just a couple of questions about that...
On 12/27/18 9:57 PM, Matthias J. Sax wrote:
All data is backed in the Kafka cluster. Data that is stored locally, is
basically a cache, and Kafka Streams will recreate the local data if you
loose it.
Thus, I am not sure how the KTable data could be stale. One possibility
might be a miss-configuration: I assume that you read the topic directly
as a table (ie, builder.table("topic")). If you do this, the used input
topic must be configured with log compaction --- if it is configured
with retention, you might loose data from the input topic and if you
also loose the local cache, Kafka Streams cannot recreate the local
state because it was deleted from the topic (log compaction will guard
the input topic from data loss).
Is it really necessary to keep the whole log topic of a particular local
store? Such log will grow indefinitely. Replay of such log will take
more and more time. Do kafka streams write just changelog to such topic
or do they eventually write a snapshot of the current store too?
If I configure 'num.standby.replicas' with number > 0, will such
replicas keep its own synchronized local store on disk? In that case
loosing an active stream processor will fallback to standby processor
and the log will only be replayed from the offset that has not been
applied yet to the local store of the stand-by processor, meaning that
we don't need to keep the whole log. But we need to be sure not to loose
all local stores of all stand-by and active processor(s) then. In case a
stand-by processor that has been chosen to be promoted to active role
does not have local store any more, the whole log will be needed again,
right?
Suppose that the store that is needed is a WindowStore which only keeps
data for a limited number of past windows. Would the "useable" part of
such store be possible to reconstruct from the limited number of past
log records so that full log would not be necessary?
Regards, Peter
-Matthias
On 12/24/18 12:22 PM, Edmondo Porcu wrote:
Hello Kafka users,
we are running a Kafka Streams as a fully stateless application, meaning
that we are not persisting /tmp/kafka-streams on a durable volume but we
are rather losing it at each restart. This application is performing a
KTable-KTable join of data coming from Kafka Connect, and sometimes we want
to force the output to tick so we update records in the right table from
the database, but we see that the left table is "stale".
Is it possible that because of reboots, the application loses some messages
? How is the state reconstructed when /tmp/kafka-streams is not available?
Is the state saved in an intermediate topic?
Thanks,
Edmondo