Ah, yes. Glad you figured it out!

Caching does not reduce EOS guarantees at all. I highly recommend using it. You 
might even want to take a look at the caching metrics to make sure you have a 
good hit ratio. 

-John

On Sat, Dec 7, 2019, at 10:51, Alessandro Tagliapietra wrote:
> Never mind I've found out I can use `.withCachingEnabled` on the store
> builder to achieve the same thing as the windowing example as
> `Materialized.as` turns that on by default.
> 
> Does caching in any way reduces the EOS guarantees?
> 
> --
> Alessandro Tagliapietra
> 
> 
> On Sat, Dec 7, 2019 at 1:12 AM Alessandro Tagliapietra <
> tagliapietra.alessan...@gmail.com> wrote:
> 
> > Seems my journey with this isn't done just yet,
> >
> > This seems very complicated to me but I'll try to explain it as best I can.
> > To better understand the streams network usage I've used prometheus with
> > the JMX exporter to export kafka metrics.
> > To check the amount of data we use I'm looking at the increments
> > of kafka_producer_topic_metrics_byte_total and
> > kafka_producer_producer_topic_metrics_record_send_total,
> >
> > Our current (before the change mentioned above) code looks like this:
> >
> > // This transformers just pairs a value with the previous one storing the
> > temporary one in a store
> > val pairsStream = metricStream
> >   .transformValues(ValueTransformerWithKeySupplier { PairTransformer() },
> > "LastValueStore")
> >   .filter { _, value: MetricSequence? -> value != null }
> >
> > // Create a store to store suppressed windows until a new one is received
> > val suppressStoreSupplier =
> > Stores.keyValueStoreBuilder(Stores.persistentKeyValueStore("suppress-store"),
> > ......
> >
> > // Window and aggregate data in 1 minute intervals
> > val aggregatedStream = pairsStream
> >   .groupByKey()
> >   .windowedBy<TimeWindow>(TimeWindows.of(Duration.ofMinutes(1)))
> >   .aggregate(
> >           { MetricSequenceList(ArrayList()) },
> >           { key, value, aggregate ->
> >               aggregate.getRecords().add(value)
> >               aggregate
> >           },
> >           Materialized.`as`<String, MetricSequenceList, WindowStore<Bytes,
> > ByteArray>>("aggregate-store").withKeySerde(Serdes.String()).withValueSerde(Settings.getValueSpecificavroSerde())
> >   )
> >   .toStream()
> >   .flatTransform(TransformerSupplier {
> >       // This transformer basically waits until a new window is received
> > to emit the previous one
> >   }, "suppress-store")
> >   .map { sensorId: String, suppressedOutput: SuppressedOutput ->
> >       .... etc ....
> >
> >
> > Basically:
> >  - all data goes through LastValueStore store that stores each message and
> > emits a pair with the previous one
> >  - the aggregate-store is used to store the per-window list of messages in
> > the aggregate method
> >  - the suppress store is used to store each received window which is
> > emitted only after a newer one is received
> >
> > What I'm experiencing is that:
> >  - during normal execution, the streams app sends to the lastvalue store
> > changelog topic 5k messages/min, the aggregate and suppress store changelog
> > topics only about 100
> >  - at some point (after many hours of operation), the streams app starts
> > sending to the aggregate and suppress store changelog topic the same amount
> > of messages going through the lastvaluestore
> >  - if I restart the streams app it goes back to the initial behavior
> >
> > You can see the behavior in this graph https://imgur.com/dJcUNSf
> > You can also see that after a restart everything goes back to normal
> > levels.
> > Regarding other metrics, process latency increases, poll latency
> > decreases, poll rate decreases, commit rate stays the same while commit
> > latency increases.
> >
> > Now, I've these questions:
> >  - why isn't the aggregate/suppress store changelog topic throughput the
> > same as the LastValueStore? Shouldn't every time it aggregates send a
> > record to the changelog?
> >  - is the windowing doing some internal caching like not sending every
> > aggregation record until the window time is passed? (if so, where can I
> > find that code since I would like to use that also for our new
> > implementation)
> >
> > Thank you in advance
> >
> > --
> > Alessandro Tagliapietra
> >
> >
> > On Wed, Dec 4, 2019 at 7:57 AM John Roesler <vvcep...@apache.org> wrote:
> >
> >> Oh, good!
> >>
> >> On Tue, Dec 3, 2019, at 23:29, Alessandro Tagliapietra wrote:
> >> > Testing on staging shows that a restart on exception is much faster and
> >> the
> >> > stream starts right away which I think means we're reading way less data
> >> > than before!
> >> >
> >> > What I was referring to is that, in Streams, the keys for window
> >> > > aggregation state is actually composed of both the window itself and
> >> the
> >> > > key. In the DSL, it looks like "Windowed<K>". That results in the
> >> store
> >> > > having a unique key per window for each K, which is why we need
> >> retention
> >> > > as well as compaction for our changelogs. But for you, if you just
> >> make the
> >> > > key "K", then compaction alone should do the trick.
> >> >
> >> > Yes we had compact,delete as cleanup policy but probably it still had a
> >> too
> >> > long retention value, also the rocksdb store is probably much faster now
> >> > having only one key per key instead of one key per window per key.
> >> >
> >> > Thanks a lot for helping! I'm now going to setup a prometheus-jmx
> >> > monitoring so we can keep better track of what's going on :)
> >> >
> >> > --
> >> > Alessandro Tagliapietra
> >> >
> >> >
> >> > On Tue, Dec 3, 2019 at 9:12 PM John Roesler <vvcep...@apache.org>
> >> wrote:
> >> >
> >> > > Oh, yeah, I remember that conversation!
> >> > >
> >> > > Yes, then, I agree, if you're only storing state of the most recent
> >> window
> >> > > for each key, and the key you use for that state is actually the key
> >> of the
> >> > > records, then an aggressive compaction policy plus your custom
> >> transformer
> >> > > seems like a good way forward.
> >> > >
> >> > > What I was referring to is that, in Streams, the keys for window
> >> > > aggregation state is actually composed of both the window itself and
> >> the
> >> > > key. In the DSL, it looks like "Windowed<K>". That results in the
> >> store
> >> > > having a unique key per window for each K, which is why we need
> >> retention
> >> > > as well as compaction for our changelogs. But for you, if you just
> >> make the
> >> > > key "K", then compaction alone should do the trick.
> >> > >
> >> > > And yes, if you manage the topic yourself, then Streams won't adjust
> >> the
> >> > > retention time. I think it might validate that the retention isn't too
> >> > > short, but I don't remember offhand.
> >> > >
> >> > > Cheers, and let me know how it goes!
> >> > > -John
> >> > >
> >> > > On Tue, Dec 3, 2019, at 23:03, Alessandro Tagliapietra wrote:
> >> > > > Hi John,
> >> > > >
> >> > > > afaik grace period uses stream time
> >> > > >
> >> > >
> >> https://kafka.apache.org/21/javadoc/org/apache/kafka/streams/kstream/Windows.html
> >> > > > which is
> >> > > > per partition, unfortunately we process data that's not in sync
> >> between
> >> > > > keys so each key needs to be independent and a key can have much
> >> older
> >> > > > data
> >> > > > than the other.
> >> > > >
> >> > > > Having a small grace period would probably close old windows sooner
> >> than
> >> > > > expected. That's also why in my use case a custom store that just
> >> stores
> >> > > > the last window data for each key might work better. I had the same
> >> issue
> >> > > > with suppression and it has been reported here
> >> > > > https://issues.apache.org/jira/browse/KAFKA-8769
> >> > > > Oh I just saw that you're the one that helped me on slack and
> >> created the
> >> > > > issue (thanks again for that).
> >> > > >
> >> > > > The behavior that you mention about streams setting changelog
> >> retention
> >> > > > time is something they do on creation of the topic when the broker
> >> has
> >> > > auto
> >> > > > creation enabled? Because we're using confluent cloud and I had to
> >> create
> >> > > > it manually.
> >> > > > Regarding the change in the recovery behavior, with compact cleanup
> >> > > policy
> >> > > > shouldn't the changelog only keep the last value? That would make
> >> the
> >> > > > recovery faster and "cheaper" as it would only need to read a single
> >> > > value
> >> > > > per key (if the cleanup just happened) right?
> >> > > >
> >> > > > --
> >> > > > Alessandro Tagliapietra
> >> > > >
> >> > > >
> >> > > > On Tue, Dec 3, 2019 at 8:51 PM John Roesler <vvcep...@apache.org>
> >> wrote:
> >> > > >
> >> > > > > Hey Alessandro,
> >> > > > >
> >> > > > > That sounds also like it would work. I'm wondering if it would
> >> actually
> >> > > > > change what you observe w.r.t. recovery behavior, though. Streams
> >> > > already
> >> > > > > sets the retention time on the changelog to equal the retention
> >> time
> >> > > of the
> >> > > > > windows, for windowed aggregations, so you shouldn't be loading a
> >> lot
> >> > > of
> >> > > > > window data for old windows you no longer care about.
> >> > > > >
> >> > > > > Have you set the "grace period" on your window definition? By
> >> default,
> >> > > it
> >> > > > > is set to 24 hours, but you can set it as low as you like. E.g.,
> >> if you
> >> > > > > want to commit to having in-order data only, then you can set the
> >> grace
> >> > > > > period to zero. This _should_ let the broker clean up the
> >> changelog
> >> > > records
> >> > > > > as soon as the window ends.
> >> > > > >
> >> > > > > Of course, the log cleaner doesn't run all the time, so there's
> >> some
> >> > > extra
> >> > > > > delay in which "expired" data would still be visible in the
> >> changelog,
> >> > > but
> >> > > > > it would actually be just the same as if you manage the store
> >> yourself.
> >> > > > >
> >> > > > > Hope this helps!
> >> > > > > -John
> >> > > > >
> >> > > > > On Tue, Dec 3, 2019, at 22:22, Alessandro Tagliapietra wrote:
> >> > > > > > Thanks John for the explanation,
> >> > > > > >
> >> > > > > > I thought that with EOS enabled (which we have) it would in the
> >> worst
> >> > > > > case
> >> > > > > > find a valid checkpoint and start the restore from there until
> >> it
> >> > > reached
> >> > > > > > the last committed status, not completely from scratch. What
> >> you say
> >> > > > > > definitely makes sense now.
> >> > > > > > Since we don't really need old time windows and we ensure data
> >> is
> >> > > ordered
> >> > > > > > when processed I think I"ll just write a custom transformer to
> >> keep
> >> > > only
> >> > > > > > the last window, store intermediate aggregation results in the
> >> store
> >> > > and
> >> > > > > > emit a new value only when we receive data belonging to a new
> >> window.
> >> > > > > > That with a compact only changelog topic should keep the rebuild
> >> > > data to
> >> > > > > > the minimum as it would have only the last value for each key.
> >> > > > > >
> >> > > > > > Hope that makes sense
> >> > > > > >
> >> > > > > > Thanks again
> >> > > > > >
> >> > > > > > --
> >> > > > > > Alessandro Tagliapietra
> >> > > > > >
> >> > > > > >
> >> > > > > > On Tue, Dec 3, 2019 at 3:04 PM John Roesler <
> >> vvcep...@apache.org>
> >> > > wrote:
> >> > > > > >
> >> > > > > > > Hi Alessandro,
> >> > > > > > >
> >> > > > > > > To take a stab at your question, maybe it first doesn't find
> >> it,
> >> > > but
> >> > > > > then
> >> > > > > > > restores some data, writes the checkpoint, and then later on,
> >> it
> >> > > has to
> >> > > > > > > re-initialize the task for some reason, and that's why it does
> >> > > find a
> >> > > > > > > checkpoint then?
> >> > > > > > >
> >> > > > > > > More to the heart of the issue, if you have EOS enabled,
> >> Streams
> >> > > _only_
> >> > > > > > > records the checkpoint when the store is in a known-consistent
> >> > > state.
> >> > > > > For
> >> > > > > > > example, if you have a graceful shutdown, Streams will flush
> >> all
> >> > > the
> >> > > > > > > stores, commit all the transactions, and then write the
> >> checkpoint
> >> > > > > file.
> >> > > > > > > Then, on re-start, it will pick up from that checkpoint.
> >> > > > > > >
> >> > > > > > > But as soon as it starts processing records, it removes the
> >> > > checkpoint
> >> > > > > > > file, so if it crashes while it was processing, there is no
> >> > > checkpoint
> >> > > > > file
> >> > > > > > > there, and it will have to restore from the beginning of the
> >> > > changelog.
> >> > > > > > >
> >> > > > > > > This design is there on purpose, because otherwise we cannot
> >> > > actually
> >> > > > > > > guarantee correctness... For example, if you are maintaining a
> >> > > count
> >> > > > > > > operation, and we process an input record i, increment the
> >> count
> >> > > and
> >> > > > > write
> >> > > > > > > it to the state store, and to the changelog topic. But we
> >> crash
> >> > > before
> >> > > > > we
> >> > > > > > > commit that transaction. Then, the write to the changelog
> >> would be
> >> > > > > aborted,
> >> > > > > > > and we would re-process record i . However, we've already
> >> updated
> >> > > the
> >> > > > > local
> >> > > > > > > state store, so when we increment it again, it results in
> >> > > > > double-counting
> >> > > > > > > i. The key point here is that there's no way to do an atomic
> >> > > operation
> >> > > > > > > across two different systems (local state store and the
> >> changelog
> >> > > > > topic).
> >> > > > > > > Since we can't guarantee that we roll back the incremented
> >> count
> >> > > when
> >> > > > > the
> >> > > > > > > changelog transaction is aborted, we can't keep the local
> >> store
> >> > > > > consistent
> >> > > > > > > with the changelog.
> >> > > > > > >
> >> > > > > > > After a crash, the only way to ensure the local store is
> >> consistent
> >> > > > > with
> >> > > > > > > the changelog is to discard the entire thing and rebuild it.
> >> This
> >> > > is
> >> > > > > why we
> >> > > > > > > have an invariant that the checkpoint file only exists when we
> >> > > _know_
> >> > > > > that
> >> > > > > > > the local store is consistent with the changelog, and this is
> >> why
> >> > > > > you're
> >> > > > > > > seeing so much bandwidth when re-starting from an unclean
> >> shutdown.
> >> > > > > > >
> >> > > > > > > Note that it's definitely possible to do better than this,
> >> and we
> >> > > would
> >> > > > > > > very much like to improve it in the future.
> >> > > > > > >
> >> > > > > > > Thanks,
> >> > > > > > > -John
> >> > > > > > >
> >> > > > > > > On Tue, Dec 3, 2019, at 16:16, Alessandro Tagliapietra wrote:
> >> > > > > > > > Hi John,
> >> > > > > > > >
> >> > > > > > > > thanks a lot for helping, regarding your message:
> >> > > > > > > >  - no we only have 1 instance of the stream application,
> >> and it
> >> > > > > always
> >> > > > > > > > re-uses the same state folder
> >> > > > > > > >  - yes we're seeing most issues when restarting not
> >> gracefully
> >> > > due
> >> > > > > > > exception
> >> > > > > > > >
> >> > > > > > > > I've enabled trace logging and filtering by a single state
> >> store
> >> > > the
> >> > > > > > > > StoreChangelogReader messages are:
> >> > > > > > > >
> >> > > > > > > > Added restorer for changelog
> >> > > > > sensors-stream-aggregate-store-changelog-0
> >> > > > > > > > Added restorer for changelog
> >> > > > > sensors-stream-aggregate-store-changelog-1
> >> > > > > > > > Added restorer for changelog
> >> > > > > sensors-stream-aggregate-store-changelog-2
> >> > > > > > > > Did not find checkpoint from changelog
> >> > > > > > > > sensors-stream-aggregate-store-changelog-2 for store
> >> > > aggregate-store,
> >> > > > > > > > rewinding to beginning.
> >> > > > > > > > Did not find checkpoint from changelog
> >> > > > > > > > sensors-stream-aggregate-store-changelog-1 for store
> >> > > aggregate-store,
> >> > > > > > > > rewinding to beginning.
> >> > > > > > > > Did not find checkpoint from changelog
> >> > > > > > > > sensors-stream-aggregate-store-changelog-0 for store
> >> > > aggregate-store,
> >> > > > > > > > rewinding to beginning.
> >> > > > > > > > No checkpoint found for task 0_2 state store aggregate-store
> >> > > > > changelog
> >> > > > > > > > sensors-stream-aggregate-store-changelog-2 with EOS turned
> >> on.
> >> > > > > > > > Reinitializing the task and restore its state from the
> >> beginning.
> >> > > > > > > > No checkpoint found for task 0_1 state store aggregate-store
> >> > > > > changelog
> >> > > > > > > > sensors-stream-aggregate-store-changelog-1 with EOS turned
> >> on.
> >> > > > > > > > Reinitializing the task and restore its state from the
> >> beginning.
> >> > > > > > > > No checkpoint found for task 0_0 state store aggregate-store
> >> > > > > changelog
> >> > > > > > > > sensors-stream-aggregate-store-changelog-0 with EOS turned
> >> on.
> >> > > > > > > > Reinitializing the task and restore its state from the
> >> beginning.
> >> > > > > > > > Found checkpoint 709937 from changelog
> >> > > > > > > > sensors-stream-aggregate-store-changelog-2 for store
> >> > > aggregate-store.
> >> > > > > > > > Restoring partition
> >> sensors-stream-aggregate-store-changelog-2
> >> > > from
> >> > > > > > > offset
> >> > > > > > > > 709937 to endOffset 742799
> >> > > > > > > > Found checkpoint 3024234 from changelog
> >> > > > > > > > sensors-stream-aggregate-store-changelog-1 for store
> >> > > aggregate-store.
> >> > > > > > > > Restoring partition
> >> sensors-stream-aggregate-store-changelog-1
> >> > > from
> >> > > > > > > offset
> >> > > > > > > > 3024234 to endOffset 3131513
> >> > > > > > > > Found checkpoint 14514072 from changelog
> >> > > > > > > > sensors-stream-aggregate-store-changelog-0 for store
> >> > > aggregate-store.
> >> > > > > > > > Restoring partition
> >> sensors-stream-aggregate-store-changelog-0
> >> > > from
> >> > > > > > > offset
> >> > > > > > > > 14514072 to endOffset 17116574
> >> > > > > > > > Restored from sensors-stream-aggregate-store-changelog-2 to
> >> > > > > > > aggregate-store
> >> > > > > > > > with 966 records, ending offset is 711432, next starting
> >> > > position is
> >> > > > > > > 711434
> >> > > > > > > > Restored from sensors-stream-aggregate-store-changelog-2 to
> >> > > > > > > aggregate-store
> >> > > > > > > > with 914 records, ending offset is 712711, next starting
> >> > > position is
> >> > > > > > > 712713
> >> > > > > > > > Restored from sensors-stream-aggregate-store-changelog-1 to
> >> > > > > > > aggregate-store
> >> > > > > > > > with 18 records, ending offset is 3024261, next starting
> >> > > position is
> >> > > > > > > 3024262
> >> > > > > > > >
> >> > > > > > > >
> >> > > > > > > > why it first says it didn't find the checkpoint and then it
> >> does
> >> > > > > find it?
> >> > > > > > > > It seems it loaded about  2.7M records (sum of offset
> >> difference
> >> > > in
> >> > > > > the
> >> > > > > > > > "restorting partition ...." messages) right?
> >> > > > > > > > Maybe should I try to reduce the checkpoint interval?
> >> > > > > > > >
> >> > > > > > > > Regards
> >> > > > > > > >
> >> > > > > > > > --
> >> > > > > > > > Alessandro Tagliapietra
> >> > > > > > > >
> >> > > > > > > >
> >> > > > > > > > On Mon, Dec 2, 2019 at 9:18 AM John Roesler <
> >> vvcep...@apache.org
> >> > > >
> >> > > > > wrote:
> >> > > > > > > >
> >> > > > > > > > > Hi Alessandro,
> >> > > > > > > > >
> >> > > > > > > > > I'm sorry to hear that.
> >> > > > > > > > >
> >> > > > > > > > > The restore process only takes one factor into account:
> >> the
> >> > > current
> >> > > > > > > offset
> >> > > > > > > > > position of the changelog topic is stored in a local file
> >> > > > > alongside the
> >> > > > > > > > > state stores. On startup, the app checks if the recorded
> >> > > position
> >> > > > > lags
> >> > > > > > > the
> >> > > > > > > > > latest offset in the changelog. If so, then it reads the
> >> > > missing
> >> > > > > > > changelog
> >> > > > > > > > > records before starting processing.
> >> > > > > > > > >
> >> > > > > > > > > Thus, it would not restore any old window data.
> >> > > > > > > > >
> >> > > > > > > > > There might be a few different things going on to explain
> >> your
> >> > > > > > > observation:
> >> > > > > > > > > * if there is more than one instance in your Streams
> >> cluster,
> >> > > > > maybe the
> >> > > > > > > > > task is "flopping" between instances, so each instance
> >> still
> >> > > has to
> >> > > > > > > recover
> >> > > > > > > > > state, since it wasn't the last one actively processing
> >> it.
> >> > > > > > > > > * if the application isn't stopped gracefully, it might
> >> not
> >> > > get a
> >> > > > > > > chance
> >> > > > > > > > > to record its offset in that local file, so on restart it
> >> has
> >> > > to
> >> > > > > > > restore
> >> > > > > > > > > some or all of the state store from changelog.
> >> > > > > > > > >
> >> > > > > > > > > Or it could be something else; that's just what comes to
> >> mind.
> >> > > > > > > > >
> >> > > > > > > > > If you want to get to the bottom of it, you can take a
> >> look at
> >> > > the
> >> > > > > > > logs,
> >> > > > > > > > > paying close attention to which tasks are assigned to
> >> which
> >> > > > > instances
> >> > > > > > > after
> >> > > > > > > > > each restart. You can also look into the logs from
> >> > > > > > > > >
> >> > > `org.apache.kafka.streams.processor.internals.StoreChangelogReader`
> >> > > > > > > (might
> >> > > > > > > > > want to set it to DEBUG or TRACE level to really see
> >> what's
> >> > > > > happening).
> >> > > > > > > > >
> >> > > > > > > > > I hope this helps!
> >> > > > > > > > > -John
> >> > > > > > > > >
> >> > > > > > > > > On Sun, Dec 1, 2019, at 21:25, Alessandro Tagliapietra
> >> wrote:
> >> > > > > > > > > > Hello everyone,
> >> > > > > > > > > >
> >> > > > > > > > > > we're having a problem with bandwidth usage on streams
> >> > > > > application
> >> > > > > > > > > startup,
> >> > > > > > > > > > our current setup does this:
> >> > > > > > > > > >
> >> > > > > > > > > > ...
> >> > > > > > > > > > .groupByKey()
> >> > > > > > > > > >
> >> > > .windowedBy<TimeWindow>(TimeWindows.of(Duration.ofMinutes(1)))
> >> > > > > > > > > > .aggregate(
> >> > > > > > > > > >         { MetricSequenceList(ArrayList()) },
> >> > > > > > > > > >         { key, value, aggregate ->
> >> > > > > > > > > >             aggregate.getRecords().add(value)
> >> > > > > > > > > >             aggregate
> >> > > > > > > > > >         },
> >> > > > > > > > > >         Materialized.`as`<String, MetricSequenceList,
> >> > > > > > > WindowStore<Bytes,
> >> > > > > > > > > >
> >> > > > > > > > >
> >> > > > > > >
> >> > > > >
> >> > >
> >> ByteArray>>("aggregate-store").withKeySerde(Serdes.String()).withValueSerde(Settings.getValueSpecificavroSerde())
> >> > > > > > > > > > )
> >> > > > > > > > > > .toStream()
> >> > > > > > > > > > .flatTransform(TransformerSupplier {
> >> > > > > > > > > > ...
> >> > > > > > > > > >
> >> > > > > > > > > > basically in each window we append the new values and
> >> then do
> >> > > > > some
> >> > > > > > > other
> >> > > > > > > > > > logic with the array of windowed values.
> >> > > > > > > > > > The aggregate-store changelog topic configuration  uses
> >> > > > > > > compact,delete as
> >> > > > > > > > > > cleanup policy and has 12 hours of retention.
> >> > > > > > > > > >
> >> > > > > > > > > > What we've seen is that on application startup it takes
> >> a
> >> > > couple
> >> > > > > > > minutes
> >> > > > > > > > > to
> >> > > > > > > > > > rebuild the state store, even if the state store
> >> directory is
> >> > > > > > > persisted
> >> > > > > > > > > > across restarts. That along with an exception that
> >> caused the
> >> > > > > docker
> >> > > > > > > > > > container to be restarted a couple hundreds times
> >> caused a
> >> > > big
> >> > > > > > > confluent
> >> > > > > > > > > > cloud bill compared to what we usually spend (1/4 of a
> >> full
> >> > > > > month in
> >> > > > > > > 1
> >> > > > > > > > > day).
> >> > > > > > > > > >
> >> > > > > > > > > > What I think is happening is that the topic is keeping
> >> all
> >> > > the
> >> > > > > > > previous
> >> > > > > > > > > > windows even with the compacting policy because each
> >> key is
> >> > > the
> >> > > > > > > original
> >> > > > > > > > > > key + the timestamp not just the key. Since we don't
> >> care
> >> > > about
> >> > > > > > > previous
> >> > > > > > > > > > windows as the flatTransform after the toStream() makes
> >> sure
> >> > > > > that we
> >> > > > > > > > > don't
> >> > > > > > > > > > process old windows (a custom suppressor basically) is
> >> there
> >> > > a
> >> > > > > way to
> >> > > > > > > > > only
> >> > > > > > > > > > keep the last window so that the store rebuilding goes
> >> > > faster and
> >> > > > > > > without
> >> > > > > > > > > > rebuilding old windows too? Or should I create a custom
> >> > > window
> >> > > > > using
> >> > > > > > > the
> >> > > > > > > > > > original key as key so that the compaction keeps only
> >> the
> >> > > last
> >> > > > > window
> >> > > > > > > > > data?
> >> > > > > > > > > >
> >> > > > > > > > > > Thank you
> >> > > > > > > > > >
> >> > > > > > > > > > --
> >> > > > > > > > > > Alessandro Tagliapietra
> >> > > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >
>

Reply via email to