Thanks  Matthias  !

my question (So, does it mean that the state store modified locally by each
application is replicated to all other applications) was mainly about
"global state store",
As I understand it from your previous response, the state for global state
store would be replicated to all instances, please correct me if wrong

On Thu, Apr 30, 2020 at 12:42 PM Matthias J. Sax <mj...@apache.org> wrote:

> > Thanks for the information. So, does it mean that the state store
> modified
> >> locally by each application is replicated to all other applications?
>
> No. As long as the application runs "normally" (ie, without failure),
> state is only maintained by one instance. In general, only in case of
> failure, state would be migrated.
>
> You can configure "standby tasks" to pro-actively maintain state on
> other instances, but it would not be on _all_ instances (depending on
> your configuration) and the main purpose of standby tasks is to reduce
> recovery time. Ie, even if you have standby task, that are "passive"
> (you could only access them via IQ)
>
>
> -Matthias
>
> On 4/29/20 9:06 PM, Pushkar Deole wrote:
> > Thanks for the information. So, does it mean that the state store
> modified
> > locally by each application is replicated to all other applications? If
> yes
> > then does it happen in real time i.e. the state modified by an
> application
> > is immediately reflected in the state store of the other application?
> >
> >
> > On Thu, Apr 30, 2020 at 12:18 AM Matthias J. Sax <mj...@apache.org>
> wrote:
> >
> >> This might help:
> >>
> >>
> https://stackoverflow.com/questions/40274884/is-kafka-stream-statestore-global-over-all-instances-or-just-local/40278168#40278168
> >>
> >> Note, that "global stores" are also "local" (ie, on local disk). The
> >> difference is really between being sharded or being replicated.
> >>
> >>
> >> -Matthias
> >>
> >> On 4/29/20 11:28 AM, Bill Bejeck wrote:
> >>> Hi Pushkar,
> >>>
> >>> There is a concept of a global store
> >>> <
> >>
> https://kafka.apache.org/25/javadoc/org/apache/kafka/streams/StreamsBuilder.html#addGlobalStore-org.apache.kafka.streams.state.StoreBuilder-java.lang.String-org.apache.kafka.streams.kstream.Consumed-org.apache.kafka.streams.processor.ProcessorSupplier-
> >>> in
> >>> Kafka Streams.
> >>> The global state store has data from all partitions from the provided
> >> input
> >>> topic, unlike a "regular" state store that only has data for the
> >> partitions
> >>> of the streams task that owns the store.
> >>> Note that global stores are still "local" because the store is
> >> materialized
> >>> on the machine running the streams app. But it's considered "global"
> >>> because it accesses all partitions of the input topic for the store.
> >>>
> >>> HTH,
> >>> Bill
> >>>
> >>> On Wed, Apr 29, 2020 at 6:09 AM Pushkar Deole <pdeole2...@gmail.com>
> >> wrote:
> >>>
> >>>> Hi,
> >>>>
> >>>> I am looking for some information as to whether kafka state store
> >>>> KeyValueStore is global or local only. I found that it can be
> in-memory
> >> or
> >>>> made persistent which stores it in local Rocksdb and also logging can
> be
> >>>> enabled so the state is backed by a topic which allows the state store
> >> to
> >>>> be fault tolerant.
> >>>> However the information that I am looking for is: whether the state
> >> store
> >>>> can be global i.e. if the state written by one application is
> available
> >> to
> >>>> the other application in real time even when both the applications are
> >>>> running i.e. no rebalancing has triggered?
> >>>>
> >>>
> >>
> >>
> >
>
>

Reply via email to