Thanks Igor for the insights!
If you feel this should be changed, feel free to open a JIRA ticket.
Best,
Bruno
On 9/6/23 9:07 PM, Igor Maznitsa wrote:
Hi Bruno
Looks like that I have found error in my code. The error was that I
split create of StoreQueryParameters and my code looked like sn
Hi Bruno
Looks like that I have found error in my code. The error was that I
split create of StoreQueryParameters and my code looked like snippet below
/*var query = StoreQueryParameters.fromNameAndType(TABLE_NAME,
queryableStoreType);*/
/*if (useStale) {*/
/* query.enableStaleStore();
*
Hi Igor,
Sorry to hear you have issues with querying standbys!
I have two questions to clarify the situation:
1. Did you enable querying stale stores with
StoreQueryParameters.fromNameAndType(TABLE_NAME,
queryableStoreType).enableStaleStores()
as described in the blog post?
2. Since you a
Hello
1. I am starting two Kafka Streams applications worked in same group
with num.standby.replicas=1
2. Application A has active TimeWindow data store and application B has
the standby version of the data store
Is there any way to read the standby store time window data in bounds of
B ap
ntion
config instead of what feels like the standard/default 7 day retention?
There is no particular reason... In fact, it's changed in 3.0.0 and
retention time is set according to "window-size + grace-period" if not
specified otherwise.
Is there a way to configure a window s
Kafka specifically adds only a day to the
window retention config instead of what feels like the standard/default 7 day
retention? Is there a way to configure a window store to be compacted only, and
Kafka Streams can just tombstone the topic when the window is closed/emitted?
I would imagine the answer t
s Guozhang.
> >
> > I don't see the remove method in window stores. Am I missing something?
> It
> > would be very nice to implement the optimization you had mentioned.
> >
> > Thanks
> >
> > On Tue, Feb 23, 2021 at 11:11 AM Guozhang Wang
> wrot
e I think your design with a KVstore plus a
> book-keeping
> > window store would work better. One minor optimization you can try
> though,
> > is that instead of trying to check if the TTL has changed or not when
> > expiring from the window store, you can try to delete from the
s a book-keeping
> window store would work better. One minor optimization you can try though,
> is that instead of trying to check if the TTL has changed or not when
> expiring from the window store, you can try to delete from the window store
> whenever you are updating the kv-store. Mor
I see. In that case I think your design with a KVstore plus a book-keeping
window store would work better. One minor optimization you can try though,
is that instead of trying to check if the TTL has changed or not when
expiring from the window store, you can try to delete from the window store
t; I have a question about how I can use window stores to achieve
> > this
> > > > use
> > > > > > > case. Thanks for all the help.
> > > > > > >
> > > > > > > A user record will be created when the user first logins and
&
gt; > > > records
> > > > > > needs to be cleaned up after 10 mins of inactivity. Thus for each
> > > user
> > > > > > there will be a TTL but the TTL value will be updated each time
> > when
> > > > the
> > > > > > user is a
t; when
> > > the
> > > > > user is active before he becomes inactive for the entire 10 min
> > period.
> > > > We
> > > > > are currently using PAPI for all our topologies and I was thinking
> of
> > > > > implementing it u
> > > implementing it using a punctuator.
> > > >
> > > > My initial logic was to have a KV store with each user as key and TTL
> > as
> > > > the value and run a scheduled task every minute that looks at all the
> > > > records which
tor.
> > >
> > > My initial logic was to have a KV store with each user as key and TTL
> as
> > > the value and run a scheduled task every minute that looks at all the
> > > records which have TTL value lesser than the timestamp. But the problem
> > in
&g
d run a scheduled task every minute that looks at all the
> > records which have TTL value lesser than the timestamp. But the problem
> in
> > this approach was performance. When there are more than 1M records it
> takes
> > more than a few seconds to complete this ta
eduled task every minute that looks at all the
> records which have TTL value lesser than the timestamp. But the problem in
> this approach was performance. When there are more than 1M records it takes
> more than a few seconds to complete this task.
>
> Next approach is to have a w
run a scheduled task every minute that looks at all the
records which have TTL value lesser than the timestamp. But the problem in
this approach was performance. When there are more than 1M records it takes
more than a few seconds to complete this task.
Next approach is to have a window store and
terialize a table with
> the updated values.
>
> It is possible to the access the new table as a KeyValueStore also, but it
> is much less convenient and I also expected that if I have a window store
> it would still be a window store after transformation/through.
>
> Thanks,
&
ected that if I have a window store
it would still be a window store after transformation/through.
Thanks,
Mikael
On Fri, Dec 16, 2016 at 6:20 PM Eno Thereska wrote:
> Hi Mikael,
>
> Currently that is not possible. Could you elaborate why you'd need that
> since you can query fro
Hi Mikael,
Currently that is not possible. Could you elaborate why you'd need that since
you can query from tableOne.
Thanks
Eno
> On 16 Dec 2016, at 10:45, Mikael Högqvist wrote:
>
> Hi,
>
> I have a small example topology that count words per minute (scala):
>
>words
> .map { (key
Hi,
I have a small example topology that count words per minute (scala):
words
.map { (key, word) =>
new KeyValue(word, Long.box(1L))
}
.groupByKey(Serdes.String, Serdes.Long)
.count(TimeWindows.of(5 * 60 * 1000L), tableOne)
.through(new WindowedSerde, Se
22 matches
Mail list logo