I created https://issues.apache.org/jira/browse/KAFKA-9501 for tracking.
On 1/30/20 10:17 AM, Matthias J. Sax wrote: > What you say is correct. This is a severe bug rendering standby tasks > useless for in-memory state stores. Can you pleas open a ticket so we > can fix it asap? > > > -Matthias > > On 1/27/20 6:05 AM, Igor Danis wrote: >> Hi all, >> >> I have question about kafka-streams, particularly in-memory state-store >> (/org.apache.kafka.streams.state.internals.InMemoryKeyValueStore/). >> >> I believe that topology is irrelevant here, but let's say I have one >> source topic with single partition >> feeding data into one statefull processor associated to single in-memory >> state store. >> This results in topology with single task. >> >> This topology is run in 2 application instances: >> - First instance (A) runs the task in active mode >> - Second instance (B) runs the task as standby >> >> Our use-case is low-latency processing, hence we need to keep rebalance >> downtime as low as possible (ideally few hundreds milliseconds). >> >> Let's say that we kill instance A, which triggers rebalance and B takes >> over the processing. >> >> We found that, when task on B transitions from STANDBY into ACTIVE mode, >> it closes in-memory state-store and effectively throws >> away any state read from changelog while it was in STANDBY. No >> checkpoints nor state is preserved. >> >> Subsequently in ACTIVE mode, it reads again changelog with >> restore-consumer. Depending on the size of the changelog this operation >> can take few minutes during which no processing is done. This happens >> despite of B having up-to-date standby-replica, >> which is really counterintuitive. What is the reason for this behavior ? >> >> Note that we initially used persistent RocksDB state-store, but we had >> simillar issues with latency (only this time it was due to RocksDB >> compaction I believe), so we prefer in-memory solution. >> >> If this question is more appropriate for developer mailing list, please >> let me know. >> >> Thanks and Regards, >> Igor >> >> >> >
signature.asc
Description: OpenPGP digital signature