You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Bruno Cadonna <br...@confluent.io> on 2019/06/04 21:09:33 UTC

Re: [DISCUSS] KIP-471: Expose RocksDB Metrics in Kafka Streams

Hi Guozhang,

After some thoughts, I tend to be in favour of the option with metrics
for each physical RocksDB instance for the following reasons:

1) A user already needs to be aware of segmented state stores when
providing a custom RocksDBConfigSetter. In RocksDBConfigSetter one can
specify settings for a store depending on the name of the store. Since
segments (i.e. state store) in a segmented state store have names that
share a prefix but have suffixes that are created at runtime, increase
with time and are theoretically unbounded, a user needs to take
account of the segments to provide the settings for all (i.e. matching
the common prefix) or some (i.e. matching the common prefix and for
example suffixes according to a specific pattern) of the segments of a
specific segmented state store.
2) Currently settings for RocksDB can only be specified by a user per
physical instance and not per logical instance. Deriving good settings
for physical instances from metrics for a logical instance can be hard
if the physical instances are not accessed uniformly. In segmented
state stores segments are not accessed uniformly.
3) Simpler to implement and to get things done.

Any thoughts on this from anybody?

Best,
Bruno

On Thu, May 30, 2019 at 8:33 PM Guozhang Wang <wa...@gmail.com> wrote:
>
> Hi Bruno:
>
> Regarding 2) I think either way has some shortcomings: exposing the metrics
> per rocksDB instance for window / session stores exposed some
> implementation internals (that we use segmented stores) to enforce users to
> be aware of them. E.g. what if we want to silently change the internal
> implementation by walking away from the segmented approach? On the other
> hand, coalescing multiple rocksDB instances' metrics into a single one per
> each logical store also has some concerns as I mentioned above. What I'm
> thinking is actually that, if we can customize the aggregation logic to
> still has one set of metrics per each logical store which may be composed
> of multiple rocksDB ones -- e.g. for `bytes-written-rate` we sum them
> across rocksDBs, while for `memtable-hit-rate` we do weighted average?
>
> Regarding logging levels, I think have DEBUG is fine, but also that means
> without manually turning it on users would not get it. Maybe we should
> consider adding some dynamic setting mechanisms in the future to allow
> users turn it on / off during run-time.
>
>
> Guozhang
>
>
>
> On Tue, May 28, 2019 at 6:23 AM Bruno Cadonna <br...@confluent.io> wrote:
>
> > Hi,
> >
> > Thank you for your comments.
> >
> > @Bill:
> >
> > 1. It is like Guozhang wrote:
> > - rocksdb-state-id is for key-value stores
> > - rocksdb-session-state-id is for session stores
> > - rocksdb-window-state-id is for window stores
> > These tags are defined in the corresponding store builders and I think
> > it is a good idea to re-use them.
> >
> > 2. I could not find any exposed ticker or histogram to get the total
> > and average number of compactions, although RocksDB dumps the number
> > of compactions between levels in its log files. There is the
> > NUM_SUBCOMPACTIONS_SCHEDULED histogram that gives you statistics about
> > the number of subcompactions actually scheduled during a compaction,
> > but that is not that what you are looking for. If they will expose the
> > number of compaction in the future, we can still add the metrics you
> > propose. I guess, the metric in this KIP that would indirectly be
> > influenced by the number of L0 files would be write-stall-duration. If
> > there are too many compactions this duration should increase. However,
> > this metric is also influenced by memtable flushes.
> >
> > @John:
> >
> > I think it is a good idea to prefix the flush-related metrics with
> > memtable to avoid ambiguity. I changed the KIP accordingly.
> >
> > @Dongjin:
> >
> > For the tag and compaction-related comments, please see my answers to Bill.
> >
> > I cannot follow your second paragraph. Are you saying that a tuning
> > guide for RocksDB within Streams based on the metrics in this KIP is
> > out of scope? I also think that it doesn't need to be included in this
> > KIP, but it is worth to work on it afterwards.
> >
> > @Guozhang:
> >
> > 1. Thank you for the explanation. I missed that. I modified the KIP
> > accordingly.
> >
> > 2. No, my plan is to record and expose the set of metrics for each
> > RocksDB store separately. Each set of metrics can then be
> > distinguished by its store ID. Do I miss something here?
> >
> > 3. I agree with you and Sophie about user education and that we should
> > work on it after this KIP.
> >
> > 4. I agree also on the user API. However, I would like to open a
> > separate KIP for it because I still need a bit of thinking to get it.
> > I also think it is a good idea to do one step after the other to get
> > at least the built-in RocksDB metrics into the next release.
> > Do you think I chose too many metrics as built-in metrics for this
> > KIP? What do others think?
> >
> > @Sophie:
> >
> > I tend to DEBUG level, but I do not have heart feelings about it. Do
> > you mean to turn it on/off RocksDB metrics in the Streams
> > configuration?
> >
> > Best,
> > Bruno
> >
> > On Tue, May 21, 2019 at 8:02 PM Sophie Blee-Goldman <so...@confluent.io>
> > wrote:
> > >
> > > I definitely agree with Guozhang's "meta" comment: if it's possible to
> > > allow users to pick and choose individual RocksDB metrics that would be
> > > ideal. One further question is whether these will be debug or info level
> > > metrics, or a separate level altogether? If there is a nontrivial
> > overhead
> > > associated with attaching RocksDB metrics it would probably be good to be
> > > able to independently turn on/off Rocks metrics
> > >
> > > On Tue, May 21, 2019 at 9:00 AM Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > > > Hello Bruno,
> > > >
> > > > Thanks for the KIP, I have a few minor comments and a meta one which
> > are
> > > > relatively aligned with other folks:
> > > >
> > > > Minor:
> > > >
> > > > 1) Regarding the "rocksdb-state-id = [store ID]", to be consistent with
> > > > other state store metrics (see
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-444%3A+Augment+metrics+for+Kafka+Streams
> > ),
> > > > this tag should be either "rocksdb-window-state-id",
> > > > "rocksdb-session-state-id" or "rocksdb-state-id". For window / session
> > > > store backed by rocksDB, the tags should not be "rocksdb-state-id".
> > > >
> > > > 2) Also for window / session store, my take is that you plan to have
> > > > multiple rocksDB behind the scene to report to the same set of
> > metrics, is
> > > > that right? My concern is that for such types of state stores, most of
> > the
> > > > access would be on the first segment rocksDB instance, and hence
> > coalescing
> > > > all of instances as a single set of metrics may dilute it.
> > > >
> > > > 3) I agree with @sophie@confluent.io <so...@confluent.io> that we
> > should
> > > > better have some documentation educating users what to do when see what
> > > > anomalies in metrics; though I think this is a long endeavoring effort
> > that
> > > > may go beyond this KIP's scope, let's keep that as a separate umbrella
> > JIRA
> > > > to accumulate knowledge over time.
> > > >
> > > >
> > > > Meta:
> > > >
> > > > 4) Instead of trying to enumerate all the ones that might be helpful,
> > I'd
> > > > recommend that we expose a user-friendly API in StreamsMetrics to allow
> > > > users to add more metrics from RocksDB they'd like to have, while only
> > > > keeping a small set of most-meaningful ones that are ubiquitously
> > useful
> > > > out-of-the-box. WDYT?
> > > >
> > > >
> > > > Guozhang
> > > >
> > > >
> > > >
> > > > On Tue, May 21, 2019 at 8:04 AM Dongjin Lee <do...@apache.org>
> > wrote:
> > > >
> > > >> Hi Bruno,
> > > >>
> > > >> I just read the KIP. I think this feature is great. As far as I know,
> > most
> > > >> Kafka users monitor the host resources, JVM resources, and Kafka
> > metrics
> > > >> only, not RocksDB for configuring the statistics feature is a little
> > bit
> > > >> tiresome. Since RocksDB impacts the performance of Kafka Streams, I
> > > >> believe
> > > >> providing these metrics with other metrics in one place is much
> > better.
> > > >>
> > > >> However, I am a little bit not assured for how much information
> > should be
> > > >> provided to the users with the documentation - how the user can
> > control
> > > >> the
> > > >>  RocksDB may on the boundary of the scope. How do you think?
> > > >>
> > > >> +1. I entirely agree to Bill's comments; I also think
> > `rocksdb-store-id`
> > > >> is
> > > >> better than `rocksdb-state-id` and metrics on total compactions and an
> > > >> average number of compactions is also needed.
> > > >>
> > > >> Regards,
> > > >> Dongjin
> > > >>
> > > >> On Tue, May 21, 2019 at 2:48 AM John Roesler <jo...@confluent.io>
> > wrote:
> > > >>
> > > >> > Hi Bruno,
> > > >> >
> > > >> > Looks really good overall. This is going to be an awesome addition.
> > > >> >
> > > >> > My only thought was that we have "bytes-flushed-(rate|total) and
> > > >> > flush-time-(avg|min|max)" metrics, and the description states that
> > > >> > these are specifically for Memtable flush operations. What do you
> > > >> > think about calling it "memtable-bytes-flushed... (etc)"? On one
> > hand,
> > > >> > I could see this being redundant, since that's the only thing that
> > > >> > gets "flushed" inside of Rocks. But on the other, we have an
> > > >> > independent "flush" operation in streams, which might be confusing.
> > > >> > Plus, it might help people who are looking at the full "menu" of
> > > >> > hundreds of metrics. They can't read and remember every description
> > > >> > while trying to understand the full list of metrics, so going for
> > > >> > maximum self-documentation in the name seems nice.
> > > >> >
> > > >> > But that's a minor thought. Modulo the others' comments, this looks
> > good
> > > >> > to me.
> > > >> >
> > > >> > Thanks,
> > > >> > -John
> > > >> >
> > > >> > On Mon, May 20, 2019 at 11:07 AM Bill Bejeck <bb...@gmail.com>
> > wrote:
> > > >> > >
> > > >> > > Hi Bruno,
> > > >> > >
> > > >> > > Thanks for the KIP, this will be a useful addition.
> > > >> > >
> > > >> > > Overall the KIP looks good to me, and I have two minor comments.
> > > >> > >
> > > >> > > 1. For the tags should, I'm wondering if rocksdb-state-id should
> > be
> > > >> > > rocksdb-store-id
> > > >> > > instead?
> > > >> > >
> > > >> > > 2. With the compaction metrics, would it be possible to add total
> > > >> > > compactions and an average number of compactions?  I've taken a
> > look
> > > >> at
> > > >> > the
> > > >> > > available RocksDB metrics, and I'm not sure.  But users can
> > control
> > > >> how
> > > >> > > many L0 files it takes to trigger compaction so if it is
> > possible; it
> > > >> may
> > > >> > > be useful.
> > > >> > >
> > > >> > > Thanks,
> > > >> > > Bill
> > > >> > >
> > > >> > >
> > > >> > > On Mon, May 20, 2019 at 9:15 AM Bruno Cadonna <bruno@confluent.io
> > >
> > > >> > wrote:
> > > >> > >
> > > >> > > > Hi Sophie,
> > > >> > > >
> > > >> > > > Thank you for your comments.
> > > >> > > >
> > > >> > > > It's a good idea to supplement the metrics with configuration
> > option
> > > >> > > > to change the metrics. I also had some thoughts about it.
> > However, I
> > > >> > > > think I need some experimentation to get this right.
> > > >> > > >
> > > >> > > > I added the block cache hit rates for index and filter blocks
> > to the
> > > >> > > > KIP. As far as I understood, they should stay at zero, if users
> > do
> > > >> not
> > > >> > > > configure RocksDB to include index and filter blocks into the
> > block
> > > >> > > > cache. Did you also understand this similarly? I guess also in
> > this
> > > >> > > > case some experimentation would be good to be sure.
> > > >> > > >
> > > >> > > > Best,
> > > >> > > > Bruno
> > > >> > > >
> > > >> > > >
> > > >> > > > On Sat, May 18, 2019 at 2:29 AM Sophie Blee-Goldman <
> > > >> > sophie@confluent.io>
> > > >> > > > wrote:
> > > >> > > > >
> > > >> > > > > Actually I wonder if it might be useful to users to be able to
> > > >> break
> > > >> > up
> > > >> > > > the
> > > >> > > > > cache hit stats by type? Some people may choose to store
> > index and
> > > >> > filter
> > > >> > > > > blocks alongside data blocks, and it would probably be very
> > > >> helpful
> > > >> > for
> > > >> > > > > them to know who is making more effective use of the cache in
> > > >> order
> > > >> > to
> > > >> > > > tune
> > > >> > > > > how much of it is allocated to each. I'm not sure how common
> > this
> > > >> > really
> > > >> > > > is
> > > >> > > > > but I think it would be invaluable to those who do. RocksDB
> > > >> > performance
> > > >> > > > can
> > > >> > > > > be quite opaque..
> > > >> > > > >
> > > >> > > > > Cheers,
> > > >> > > > > Sophie
> > > >> > > > >
> > > >> > > > > On Fri, May 17, 2019 at 5:01 PM Sophie Blee-Goldman <
> > > >> > sophie@confluent.io
> > > >> > > > >
> > > >> > > > > wrote:
> > > >> > > > >
> > > >> > > > > > Hey Bruno!
> > > >> > > > > >
> > > >> > > > > > This all looks pretty good to me, but one suggestion I have
> > is
> > > >> to
> > > >> > > > > > supplement each of the metrics with some info on how the
> > user
> > > >> can
> > > >> > > > control
> > > >> > > > > > them. In other words, which options could/should they set in
> > > >> > > > > > RocksDBConfigSetter should they discover a particular
> > > >> bottleneck?
> > > >> > > > > >
> > > >> > > > > > I don't think this necessarily needs to go into the KIP,
> > but I
> > > >> do
> > > >> > > > think it
> > > >> > > > > > should be included in the docs somewhere (happy to help
> > build up
> > > >> > the
> > > >> > > > list
> > > >> > > > > > of associated options when the time comes)
> > > >> > > > > >
> > > >> > > > > > Thanks!
> > > >> > > > > > Sophie
> > > >> > > > > >
> > > >> > > > > > On Fri, May 17, 2019 at 2:54 PM Bruno Cadonna <
> > > >> bruno@confluent.io>
> > > >> > > > wrote:
> > > >> > > > > >
> > > >> > > > > >> Hi all,
> > > >> > > > > >>
> > > >> > > > > >> this KIP describes the extension of the Kafka Streams'
> > metrics
> > > >> to
> > > >> > > > include
> > > >> > > > > >> RocksDB's internal statistics.
> > > >> > > > > >>
> > > >> > > > > >> Please have a look at it and let me know what you think.
> > Since
> > > >> I
> > > >> > am
> > > >> > > > not a
> > > >> > > > > >> RocksDB expert, I am thankful for any additional pair of
> > eyes
> > > >> that
> > > >> > > > > >> evaluates this KIP.
> > > >> > > > > >>
> > > >> > > > > >>
> > > >> > > > > >>
> > > >> > > >
> > > >> >
> > > >>
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-471:+Expose+RocksDB+Metrics+in+Kafka+Streams
> > > >> > > > > >>
> > > >> > > > > >> Best regards,
> > > >> > > > > >> Bruno
> > > >> > > > > >>
> > > >> > > > > >
> > > >> > > >
> > > >> >
> > > >>
> > > >>
> > > >> --
> > > >> *Dongjin Lee*
> > > >>
> > > >> *A hitchhiker in the mathematical world.*
> > > >> *github:  <http://goog_969573159/>github.com/dongjinleekr
> > > >> <https://github.com/dongjinleekr>linkedin:
> > > >> kr.linkedin.com/in/dongjinleekr
> > > >> <https://kr.linkedin.com/in/dongjinleekr>speakerdeck:
> > > >> speakerdeck.com/dongjin
> > > >> <https://speakerdeck.com/dongjin>*
> > > >>
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> >
>
>
> --
> -- Guozhang

Re: [DISCUSS] KIP-471: Expose RocksDB Metrics in Kafka Streams

Posted by Sophie Blee-Goldman <so...@confluent.io>.
I'm not sure we can safely assume only the most recent segment is hot.
Anything within the current window size is still being actively queried,
and users can independently set windowSize and retentionPeriod as long as
windowSize <= retentionPeriod. But the default segmentInterval is
max(retentionPeriod / 2, 60,000 ms). So if a store had windowSize ≈
retentionPeriod then the current window actually spans the latest two or
three stores. We could consider just providing metrics for however many
stores fit within windowSize but that will basically be either all/most of
them, or leave out anything being queried by IQ.

Of course the IQ access patterns are probably fairly different so there's
some question of whether they should be included in the metrics, but I
don't see how you could get rocksdb to distinguish which reads are coming
from Streams and which are IQ. You could trim some of them by only
reporting metrics for stores within the windowSize as mentioned above, but
then you'd still be mixing in IQ that just happened to be in the latest
stores. It's probably better to include all IQ or none, and I'm not sure
how you could accomplish none.

On Thu, Jun 6, 2019 at 12:14 PM Bruno Cadonna <br...@confluent.io> wrote:

> Hi,
>
> I like the idea of just exposing the metrics of the latest segment. I
> think it gives the most realistic picture of the current operations on
> the segmented RocksDB without exposing implementation details. The
> cons of this approach is that during the switch to a new segment the
> values of some metrics might behave a bit strangely because of the
> empty memtable and empty caches of the new segment. I will follow up
> on this idea and see what I need to change in the KIP.
>
> Best,
> Bruno
>
>
>
>
> On Thu, Jun 6, 2019 at 2:33 AM Guozhang Wang <wa...@gmail.com> wrote:
> >
> > I think Bruno's 2) is that for a segmented store, the access rate on
> > different segments will very likely be different. And in fact, most of
> the
> > access should be on the "latest" segment unless 1) very late arrived
> data,
> > which should be captured on the higher-level `lateness` metrics already,
> > and 2) IQ reads on old windows. The problem is that, say if 99% of reads
> go
> > to the latest segment, and 1% goes to rest of the segments, how should
> > `memtable-hit-rate` be calculated then.
> >
> > Another wild thought just to throw here: maybe we can just expose the
> > latest segment's state store as the logical store's metrics? Admittedly
> it
> > would not be most accurate, but it is 1) future-proof if we want to
> > consolidate to 1-1 physical store -> logical store implementation, and 2)
> > it is as simple and not needing to bookkeep older segments who should be
> > rarely accessed. My question is though, if upon segment rolling our
> metrics
> > can be auto-switched to the new store.
> >
> >
> > Guozhang
> >
> > On Tue, Jun 4, 2019 at 3:06 PM Sophie Blee-Goldman <so...@confluent.io>
> > wrote:
> >
> > > Hey Bruno,
> > >
> > > I tend to agree with Guozhang on this matter although you do bring up
> some
> > > good points that should be addressed. Regarding 1) I think it is
> probably
> > > fairly uncommon in practice for users to leverage the individual store
> > > names passed to RocksDBConfigSetter#setConfig in order to specify
> options
> > > on a per-store basis. When this actually is used, it does seem likely
> that
> > > users would be doing something like pattern matching the physical store
> > > name prefix in order to apply configs to all physical stores (segments)
> > > within a single logical RocksDBStore. As you mention this is something
> of a
> > > hassle already as physical stores are created/deleted, while most
> likely
> > > all anyone cares about is the prefix corresponding to the logical
> store. It
> > > seems like rather than persist this hassle to the metric layer, we
> should
> > > consider refactoring RocksDBConfigSetter to apply to a logical store
> rather
> > > than a specific physical segment. Or maybe providing some kind of
> tooling
> > > to at least make this easier on users, but that's definitely outside
> the
> > > scope of this KIP.
> > >
> > > Regarding 2) can you clarify your point about accessing stores
> uniformly?
> > > While I agree there will definitely be variance in the access pattern
> of
> > > different segments, I think it's unlikely that it will vary in any
> kind of
> > > predictable or deterministic way, hence it is not that useful to know
> in
> > > hindsight the difference reflected by the metrics.
> > >
> > > Cheers,
> > > Sophie
> > >
> > > On Tue, Jun 4, 2019 at 2:09 PM Bruno Cadonna <br...@confluent.io>
> wrote:
> > >
> > > > Hi Guozhang,
> > > >
> > > > After some thoughts, I tend to be in favour of the option with
> metrics
> > > > for each physical RocksDB instance for the following reasons:
> > > >
> > > > 1) A user already needs to be aware of segmented state stores when
> > > > providing a custom RocksDBConfigSetter. In RocksDBConfigSetter one
> can
> > > > specify settings for a store depending on the name of the store.
> Since
> > > > segments (i.e. state store) in a segmented state store have names
> that
> > > > share a prefix but have suffixes that are created at runtime,
> increase
> > > > with time and are theoretically unbounded, a user needs to take
> > > > account of the segments to provide the settings for all (i.e.
> matching
> > > > the common prefix) or some (i.e. matching the common prefix and for
> > > > example suffixes according to a specific pattern) of the segments of
> a
> > > > specific segmented state store.
> > > > 2) Currently settings for RocksDB can only be specified by a user per
> > > > physical instance and not per logical instance. Deriving good
> settings
> > > > for physical instances from metrics for a logical instance can be
> hard
> > > > if the physical instances are not accessed uniformly. In segmented
> > > > state stores segments are not accessed uniformly.
> > > > 3) Simpler to implement and to get things done.
> > > >
> > > > Any thoughts on this from anybody?
> > > >
> > > > Best,
> > > > Bruno
> > > >
> > > > On Thu, May 30, 2019 at 8:33 PM Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > > >
> > > > > Hi Bruno:
> > > > >
> > > > > Regarding 2) I think either way has some shortcomings: exposing the
> > > > metrics
> > > > > per rocksDB instance for window / session stores exposed some
> > > > > implementation internals (that we use segmented stores) to enforce
> > > users
> > > > to
> > > > > be aware of them. E.g. what if we want to silently change the
> internal
> > > > > implementation by walking away from the segmented approach? On the
> > > other
> > > > > hand, coalescing multiple rocksDB instances' metrics into a single
> one
> > > > per
> > > > > each logical store also has some concerns as I mentioned above.
> What
> > > I'm
> > > > > thinking is actually that, if we can customize the aggregation
> logic to
> > > > > still has one set of metrics per each logical store which may be
> > > composed
> > > > > of multiple rocksDB ones -- e.g. for `bytes-written-rate` we sum
> them
> > > > > across rocksDBs, while for `memtable-hit-rate` we do weighted
> average?
> > > > >
> > > > > Regarding logging levels, I think have DEBUG is fine, but also that
> > > means
> > > > > without manually turning it on users would not get it. Maybe we
> should
> > > > > consider adding some dynamic setting mechanisms in the future to
> allow
> > > > > users turn it on / off during run-time.
> > > > >
> > > > >
> > > > > Guozhang
> > > > >
> > > > >
> > > > >
> > > > > On Tue, May 28, 2019 at 6:23 AM Bruno Cadonna <br...@confluent.io>
> > > > wrote:
> > > > >
> > > > > > Hi,
> > > > > >
> > > > > > Thank you for your comments.
> > > > > >
> > > > > > @Bill:
> > > > > >
> > > > > > 1. It is like Guozhang wrote:
> > > > > > - rocksdb-state-id is for key-value stores
> > > > > > - rocksdb-session-state-id is for session stores
> > > > > > - rocksdb-window-state-id is for window stores
> > > > > > These tags are defined in the corresponding store builders and I
> > > think
> > > > > > it is a good idea to re-use them.
> > > > > >
> > > > > > 2. I could not find any exposed ticker or histogram to get the
> total
> > > > > > and average number of compactions, although RocksDB dumps the
> number
> > > > > > of compactions between levels in its log files. There is the
> > > > > > NUM_SUBCOMPACTIONS_SCHEDULED histogram that gives you statistics
> > > about
> > > > > > the number of subcompactions actually scheduled during a
> compaction,
> > > > > > but that is not that what you are looking for. If they will
> expose
> > > the
> > > > > > number of compaction in the future, we can still add the metrics
> you
> > > > > > propose. I guess, the metric in this KIP that would indirectly be
> > > > > > influenced by the number of L0 files would be
> write-stall-duration.
> > > If
> > > > > > there are too many compactions this duration should increase.
> > > However,
> > > > > > this metric is also influenced by memtable flushes.
> > > > > >
> > > > > > @John:
> > > > > >
> > > > > > I think it is a good idea to prefix the flush-related metrics
> with
> > > > > > memtable to avoid ambiguity. I changed the KIP accordingly.
> > > > > >
> > > > > > @Dongjin:
> > > > > >
> > > > > > For the tag and compaction-related comments, please see my
> answers to
> > > > Bill.
> > > > > >
> > > > > > I cannot follow your second paragraph. Are you saying that a
> tuning
> > > > > > guide for RocksDB within Streams based on the metrics in this
> KIP is
> > > > > > out of scope? I also think that it doesn't need to be included in
> > > this
> > > > > > KIP, but it is worth to work on it afterwards.
> > > > > >
> > > > > > @Guozhang:
> > > > > >
> > > > > > 1. Thank you for the explanation. I missed that. I modified the
> KIP
> > > > > > accordingly.
> > > > > >
> > > > > > 2. No, my plan is to record and expose the set of metrics for
> each
> > > > > > RocksDB store separately. Each set of metrics can then be
> > > > > > distinguished by its store ID. Do I miss something here?
> > > > > >
> > > > > > 3. I agree with you and Sophie about user education and that we
> > > should
> > > > > > work on it after this KIP.
> > > > > >
> > > > > > 4. I agree also on the user API. However, I would like to open a
> > > > > > separate KIP for it because I still need a bit of thinking to
> get it.
> > > > > > I also think it is a good idea to do one step after the other to
> get
> > > > > > at least the built-in RocksDB metrics into the next release.
> > > > > > Do you think I chose too many metrics as built-in metrics for
> this
> > > > > > KIP? What do others think?
> > > > > >
> > > > > > @Sophie:
> > > > > >
> > > > > > I tend to DEBUG level, but I do not have heart feelings about
> it. Do
> > > > > > you mean to turn it on/off RocksDB metrics in the Streams
> > > > > > configuration?
> > > > > >
> > > > > > Best,
> > > > > > Bruno
> > > > > >
> > > > > > On Tue, May 21, 2019 at 8:02 PM Sophie Blee-Goldman <
> > > > sophie@confluent.io>
> > > > > > wrote:
> > > > > > >
> > > > > > > I definitely agree with Guozhang's "meta" comment: if it's
> possible
> > > > to
> > > > > > > allow users to pick and choose individual RocksDB metrics that
> > > would
> > > > be
> > > > > > > ideal. One further question is whether these will be debug or
> info
> > > > level
> > > > > > > metrics, or a separate level altogether? If there is a
> nontrivial
> > > > > > overhead
> > > > > > > associated with attaching RocksDB metrics it would probably be
> good
> > > > to be
> > > > > > > able to independently turn on/off Rocks metrics
> > > > > > >
> > > > > > > On Tue, May 21, 2019 at 9:00 AM Guozhang Wang <
> wangguoz@gmail.com>
> > > > > > wrote:
> > > > > > >
> > > > > > > > Hello Bruno,
> > > > > > > >
> > > > > > > > Thanks for the KIP, I have a few minor comments and a meta
> one
> > > > which
> > > > > > are
> > > > > > > > relatively aligned with other folks:
> > > > > > > >
> > > > > > > > Minor:
> > > > > > > >
> > > > > > > > 1) Regarding the "rocksdb-state-id = [store ID]", to be
> > > consistent
> > > > with
> > > > > > > > other state store metrics (see
> > > > > > > >
> > > > > >
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-444%3A+Augment+metrics+for+Kafka+Streams
> > > > > > ),
> > > > > > > > this tag should be either "rocksdb-window-state-id",
> > > > > > > > "rocksdb-session-state-id" or "rocksdb-state-id". For window
> /
> > > > session
> > > > > > > > store backed by rocksDB, the tags should not be
> > > "rocksdb-state-id".
> > > > > > > >
> > > > > > > > 2) Also for window / session store, my take is that you plan
> to
> > > > have
> > > > > > > > multiple rocksDB behind the scene to report to the same set
> of
> > > > > > metrics, is
> > > > > > > > that right? My concern is that for such types of state
> stores,
> > > > most of
> > > > > > the
> > > > > > > > access would be on the first segment rocksDB instance, and
> hence
> > > > > > coalescing
> > > > > > > > all of instances as a single set of metrics may dilute it.
> > > > > > > >
> > > > > > > > 3) I agree with @sophie@confluent.io <so...@confluent.io>
> that
> > > we
> > > > > > should
> > > > > > > > better have some documentation educating users what to do
> when
> > > see
> > > > what
> > > > > > > > anomalies in metrics; though I think this is a long
> endeavoring
> > > > effort
> > > > > > that
> > > > > > > > may go beyond this KIP's scope, let's keep that as a separate
> > > > umbrella
> > > > > > JIRA
> > > > > > > > to accumulate knowledge over time.
> > > > > > > >
> > > > > > > >
> > > > > > > > Meta:
> > > > > > > >
> > > > > > > > 4) Instead of trying to enumerate all the ones that might be
> > > > helpful,
> > > > > > I'd
> > > > > > > > recommend that we expose a user-friendly API in
> StreamsMetrics to
> > > > allow
> > > > > > > > users to add more metrics from RocksDB they'd like to have,
> while
> > > > only
> > > > > > > > keeping a small set of most-meaningful ones that are
> ubiquitously
> > > > > > useful
> > > > > > > > out-of-the-box. WDYT?
> > > > > > > >
> > > > > > > >
> > > > > > > > Guozhang
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > On Tue, May 21, 2019 at 8:04 AM Dongjin Lee <
> dongjin@apache.org>
> > > > > > wrote:
> > > > > > > >
> > > > > > > >> Hi Bruno,
> > > > > > > >>
> > > > > > > >> I just read the KIP. I think this feature is great. As far
> as I
> > > > know,
> > > > > > most
> > > > > > > >> Kafka users monitor the host resources, JVM resources, and
> Kafka
> > > > > > metrics
> > > > > > > >> only, not RocksDB for configuring the statistics feature is
> a
> > > > little
> > > > > > bit
> > > > > > > >> tiresome. Since RocksDB impacts the performance of Kafka
> > > Streams,
> > > > I
> > > > > > > >> believe
> > > > > > > >> providing these metrics with other metrics in one place is
> much
> > > > > > better.
> > > > > > > >>
> > > > > > > >> However, I am a little bit not assured for how much
> information
> > > > > > should be
> > > > > > > >> provided to the users with the documentation - how the user
> can
> > > > > > control
> > > > > > > >> the
> > > > > > > >>  RocksDB may on the boundary of the scope. How do you think?
> > > > > > > >>
> > > > > > > >> +1. I entirely agree to Bill's comments; I also think
> > > > > > `rocksdb-store-id`
> > > > > > > >> is
> > > > > > > >> better than `rocksdb-state-id` and metrics on total
> compactions
> > > > and an
> > > > > > > >> average number of compactions is also needed.
> > > > > > > >>
> > > > > > > >> Regards,
> > > > > > > >> Dongjin
> > > > > > > >>
> > > > > > > >> On Tue, May 21, 2019 at 2:48 AM John Roesler <
> john@confluent.io
> > > >
> > > > > > wrote:
> > > > > > > >>
> > > > > > > >> > Hi Bruno,
> > > > > > > >> >
> > > > > > > >> > Looks really good overall. This is going to be an awesome
> > > > addition.
> > > > > > > >> >
> > > > > > > >> > My only thought was that we have
> "bytes-flushed-(rate|total)
> > > and
> > > > > > > >> > flush-time-(avg|min|max)" metrics, and the description
> states
> > > > that
> > > > > > > >> > these are specifically for Memtable flush operations.
> What do
> > > > you
> > > > > > > >> > think about calling it "memtable-bytes-flushed... (etc)"?
> On
> > > one
> > > > > > hand,
> > > > > > > >> > I could see this being redundant, since that's the only
> thing
> > > > that
> > > > > > > >> > gets "flushed" inside of Rocks. But on the other, we have
> an
> > > > > > > >> > independent "flush" operation in streams, which might be
> > > > confusing.
> > > > > > > >> > Plus, it might help people who are looking at the full
> "menu"
> > > of
> > > > > > > >> > hundreds of metrics. They can't read and remember every
> > > > description
> > > > > > > >> > while trying to understand the full list of metrics, so
> going
> > > > for
> > > > > > > >> > maximum self-documentation in the name seems nice.
> > > > > > > >> >
> > > > > > > >> > But that's a minor thought. Modulo the others' comments,
> this
> > > > looks
> > > > > > good
> > > > > > > >> > to me.
> > > > > > > >> >
> > > > > > > >> > Thanks,
> > > > > > > >> > -John
> > > > > > > >> >
> > > > > > > >> > On Mon, May 20, 2019 at 11:07 AM Bill Bejeck <
> > > bbejeck@gmail.com
> > > > >
> > > > > > wrote:
> > > > > > > >> > >
> > > > > > > >> > > Hi Bruno,
> > > > > > > >> > >
> > > > > > > >> > > Thanks for the KIP, this will be a useful addition.
> > > > > > > >> > >
> > > > > > > >> > > Overall the KIP looks good to me, and I have two minor
> > > > comments.
> > > > > > > >> > >
> > > > > > > >> > > 1. For the tags should, I'm wondering if
> rocksdb-state-id
> > > > should
> > > > > > be
> > > > > > > >> > > rocksdb-store-id
> > > > > > > >> > > instead?
> > > > > > > >> > >
> > > > > > > >> > > 2. With the compaction metrics, would it be possible to
> add
> > > > total
> > > > > > > >> > > compactions and an average number of compactions?  I've
> > > taken
> > > > a
> > > > > > look
> > > > > > > >> at
> > > > > > > >> > the
> > > > > > > >> > > available RocksDB metrics, and I'm not sure.  But users
> can
> > > > > > control
> > > > > > > >> how
> > > > > > > >> > > many L0 files it takes to trigger compaction so if it is
> > > > > > possible; it
> > > > > > > >> may
> > > > > > > >> > > be useful.
> > > > > > > >> > >
> > > > > > > >> > > Thanks,
> > > > > > > >> > > Bill
> > > > > > > >> > >
> > > > > > > >> > >
> > > > > > > >> > > On Mon, May 20, 2019 at 9:15 AM Bruno Cadonna <
> > > > bruno@confluent.io
> > > > > > >
> > > > > > > >> > wrote:
> > > > > > > >> > >
> > > > > > > >> > > > Hi Sophie,
> > > > > > > >> > > >
> > > > > > > >> > > > Thank you for your comments.
> > > > > > > >> > > >
> > > > > > > >> > > > It's a good idea to supplement the metrics with
> > > > configuration
> > > > > > option
> > > > > > > >> > > > to change the metrics. I also had some thoughts about
> it.
> > > > > > However, I
> > > > > > > >> > > > think I need some experimentation to get this right.
> > > > > > > >> > > >
> > > > > > > >> > > > I added the block cache hit rates for index and filter
> > > > blocks
> > > > > > to the
> > > > > > > >> > > > KIP. As far as I understood, they should stay at
> zero, if
> > > > users
> > > > > > do
> > > > > > > >> not
> > > > > > > >> > > > configure RocksDB to include index and filter blocks
> into
> > > > the
> > > > > > block
> > > > > > > >> > > > cache. Did you also understand this similarly? I guess
> > > also
> > > > in
> > > > > > this
> > > > > > > >> > > > case some experimentation would be good to be sure.
> > > > > > > >> > > >
> > > > > > > >> > > > Best,
> > > > > > > >> > > > Bruno
> > > > > > > >> > > >
> > > > > > > >> > > >
> > > > > > > >> > > > On Sat, May 18, 2019 at 2:29 AM Sophie Blee-Goldman <
> > > > > > > >> > sophie@confluent.io>
> > > > > > > >> > > > wrote:
> > > > > > > >> > > > >
> > > > > > > >> > > > > Actually I wonder if it might be useful to users to
> be
> > > > able to
> > > > > > > >> break
> > > > > > > >> > up
> > > > > > > >> > > > the
> > > > > > > >> > > > > cache hit stats by type? Some people may choose to
> store
> > > > > > index and
> > > > > > > >> > filter
> > > > > > > >> > > > > blocks alongside data blocks, and it would probably
> be
> > > > very
> > > > > > > >> helpful
> > > > > > > >> > for
> > > > > > > >> > > > > them to know who is making more effective use of the
> > > > cache in
> > > > > > > >> order
> > > > > > > >> > to
> > > > > > > >> > > > tune
> > > > > > > >> > > > > how much of it is allocated to each. I'm not sure
> how
> > > > common
> > > > > > this
> > > > > > > >> > really
> > > > > > > >> > > > is
> > > > > > > >> > > > > but I think it would be invaluable to those who do.
> > > > RocksDB
> > > > > > > >> > performance
> > > > > > > >> > > > can
> > > > > > > >> > > > > be quite opaque..
> > > > > > > >> > > > >
> > > > > > > >> > > > > Cheers,
> > > > > > > >> > > > > Sophie
> > > > > > > >> > > > >
> > > > > > > >> > > > > On Fri, May 17, 2019 at 5:01 PM Sophie Blee-Goldman
> <
> > > > > > > >> > sophie@confluent.io
> > > > > > > >> > > > >
> > > > > > > >> > > > > wrote:
> > > > > > > >> > > > >
> > > > > > > >> > > > > > Hey Bruno!
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > This all looks pretty good to me, but one
> suggestion I
> > > > have
> > > > > > is
> > > > > > > >> to
> > > > > > > >> > > > > > supplement each of the metrics with some info on
> how
> > > the
> > > > > > user
> > > > > > > >> can
> > > > > > > >> > > > control
> > > > > > > >> > > > > > them. In other words, which options could/should
> they
> > > > set in
> > > > > > > >> > > > > > RocksDBConfigSetter should they discover a
> particular
> > > > > > > >> bottleneck?
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > I don't think this necessarily needs to go into
> the
> > > KIP,
> > > > > > but I
> > > > > > > >> do
> > > > > > > >> > > > think it
> > > > > > > >> > > > > > should be included in the docs somewhere (happy to
> > > help
> > > > > > build up
> > > > > > > >> > the
> > > > > > > >> > > > list
> > > > > > > >> > > > > > of associated options when the time comes)
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > Thanks!
> > > > > > > >> > > > > > Sophie
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > On Fri, May 17, 2019 at 2:54 PM Bruno Cadonna <
> > > > > > > >> bruno@confluent.io>
> > > > > > > >> > > > wrote:
> > > > > > > >> > > > > >
> > > > > > > >> > > > > >> Hi all,
> > > > > > > >> > > > > >>
> > > > > > > >> > > > > >> this KIP describes the extension of the Kafka
> > > Streams'
> > > > > > metrics
> > > > > > > >> to
> > > > > > > >> > > > include
> > > > > > > >> > > > > >> RocksDB's internal statistics.
> > > > > > > >> > > > > >>
> > > > > > > >> > > > > >> Please have a look at it and let me know what you
> > > > think.
> > > > > > Since
> > > > > > > >> I
> > > > > > > >> > am
> > > > > > > >> > > > not a
> > > > > > > >> > > > > >> RocksDB expert, I am thankful for any additional
> pair
> > > > of
> > > > > > eyes
> > > > > > > >> that
> > > > > > > >> > > > > >> evaluates this KIP.
> > > > > > > >> > > > > >>
> > > > > > > >> > > > > >>
> > > > > > > >> > > > > >>
> > > > > > > >> > > >
> > > > > > > >> >
> > > > > > > >>
> > > > > >
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-471:+Expose+RocksDB+Metrics+in+Kafka+Streams
> > > > > > > >> > > > > >>
> > > > > > > >> > > > > >> Best regards,
> > > > > > > >> > > > > >> Bruno
> > > > > > > >> > > > > >>
> > > > > > > >> > > > > >
> > > > > > > >> > > >
> > > > > > > >> >
> > > > > > > >>
> > > > > > > >>
> > > > > > > >> --
> > > > > > > >> *Dongjin Lee*
> > > > > > > >>
> > > > > > > >> *A hitchhiker in the mathematical world.*
> > > > > > > >> *github:  <http://goog_969573159/>github.com/dongjinleekr
> > > > > > > >> <https://github.com/dongjinleekr>linkedin:
> > > > > > > >> kr.linkedin.com/in/dongjinleekr
> > > > > > > >> <https://kr.linkedin.com/in/dongjinleekr>speakerdeck:
> > > > > > > >> speakerdeck.com/dongjin
> > > > > > > >> <https://speakerdeck.com/dongjin>*
> > > > > > > >>
> > > > > > > >
> > > > > > > >
> > > > > > > > --
> > > > > > > > -- Guozhang
> > > > > > > >
> > > > > >
> > > > >
> > > > >
> > > > > --
> > > > > -- Guozhang
> > > >
> > >
> >
> >
> > --
> > -- Guozhang
>

Re: [DISCUSS] KIP-471: Expose RocksDB Metrics in Kafka Streams

Posted by Bruno Cadonna <br...@confluent.io>.
Hi,

I like the idea of just exposing the metrics of the latest segment. I
think it gives the most realistic picture of the current operations on
the segmented RocksDB without exposing implementation details. The
cons of this approach is that during the switch to a new segment the
values of some metrics might behave a bit strangely because of the
empty memtable and empty caches of the new segment. I will follow up
on this idea and see what I need to change in the KIP.

Best,
Bruno




On Thu, Jun 6, 2019 at 2:33 AM Guozhang Wang <wa...@gmail.com> wrote:
>
> I think Bruno's 2) is that for a segmented store, the access rate on
> different segments will very likely be different. And in fact, most of the
> access should be on the "latest" segment unless 1) very late arrived data,
> which should be captured on the higher-level `lateness` metrics already,
> and 2) IQ reads on old windows. The problem is that, say if 99% of reads go
> to the latest segment, and 1% goes to rest of the segments, how should
> `memtable-hit-rate` be calculated then.
>
> Another wild thought just to throw here: maybe we can just expose the
> latest segment's state store as the logical store's metrics? Admittedly it
> would not be most accurate, but it is 1) future-proof if we want to
> consolidate to 1-1 physical store -> logical store implementation, and 2)
> it is as simple and not needing to bookkeep older segments who should be
> rarely accessed. My question is though, if upon segment rolling our metrics
> can be auto-switched to the new store.
>
>
> Guozhang
>
> On Tue, Jun 4, 2019 at 3:06 PM Sophie Blee-Goldman <so...@confluent.io>
> wrote:
>
> > Hey Bruno,
> >
> > I tend to agree with Guozhang on this matter although you do bring up some
> > good points that should be addressed. Regarding 1) I think it is probably
> > fairly uncommon in practice for users to leverage the individual store
> > names passed to RocksDBConfigSetter#setConfig in order to specify options
> > on a per-store basis. When this actually is used, it does seem likely that
> > users would be doing something like pattern matching the physical store
> > name prefix in order to apply configs to all physical stores (segments)
> > within a single logical RocksDBStore. As you mention this is something of a
> > hassle already as physical stores are created/deleted, while most likely
> > all anyone cares about is the prefix corresponding to the logical store. It
> > seems like rather than persist this hassle to the metric layer, we should
> > consider refactoring RocksDBConfigSetter to apply to a logical store rather
> > than a specific physical segment. Or maybe providing some kind of tooling
> > to at least make this easier on users, but that's definitely outside the
> > scope of this KIP.
> >
> > Regarding 2) can you clarify your point about accessing stores uniformly?
> > While I agree there will definitely be variance in the access pattern of
> > different segments, I think it's unlikely that it will vary in any kind of
> > predictable or deterministic way, hence it is not that useful to know in
> > hindsight the difference reflected by the metrics.
> >
> > Cheers,
> > Sophie
> >
> > On Tue, Jun 4, 2019 at 2:09 PM Bruno Cadonna <br...@confluent.io> wrote:
> >
> > > Hi Guozhang,
> > >
> > > After some thoughts, I tend to be in favour of the option with metrics
> > > for each physical RocksDB instance for the following reasons:
> > >
> > > 1) A user already needs to be aware of segmented state stores when
> > > providing a custom RocksDBConfigSetter. In RocksDBConfigSetter one can
> > > specify settings for a store depending on the name of the store. Since
> > > segments (i.e. state store) in a segmented state store have names that
> > > share a prefix but have suffixes that are created at runtime, increase
> > > with time and are theoretically unbounded, a user needs to take
> > > account of the segments to provide the settings for all (i.e. matching
> > > the common prefix) or some (i.e. matching the common prefix and for
> > > example suffixes according to a specific pattern) of the segments of a
> > > specific segmented state store.
> > > 2) Currently settings for RocksDB can only be specified by a user per
> > > physical instance and not per logical instance. Deriving good settings
> > > for physical instances from metrics for a logical instance can be hard
> > > if the physical instances are not accessed uniformly. In segmented
> > > state stores segments are not accessed uniformly.
> > > 3) Simpler to implement and to get things done.
> > >
> > > Any thoughts on this from anybody?
> > >
> > > Best,
> > > Bruno
> > >
> > > On Thu, May 30, 2019 at 8:33 PM Guozhang Wang <wa...@gmail.com>
> > wrote:
> > > >
> > > > Hi Bruno:
> > > >
> > > > Regarding 2) I think either way has some shortcomings: exposing the
> > > metrics
> > > > per rocksDB instance for window / session stores exposed some
> > > > implementation internals (that we use segmented stores) to enforce
> > users
> > > to
> > > > be aware of them. E.g. what if we want to silently change the internal
> > > > implementation by walking away from the segmented approach? On the
> > other
> > > > hand, coalescing multiple rocksDB instances' metrics into a single one
> > > per
> > > > each logical store also has some concerns as I mentioned above. What
> > I'm
> > > > thinking is actually that, if we can customize the aggregation logic to
> > > > still has one set of metrics per each logical store which may be
> > composed
> > > > of multiple rocksDB ones -- e.g. for `bytes-written-rate` we sum them
> > > > across rocksDBs, while for `memtable-hit-rate` we do weighted average?
> > > >
> > > > Regarding logging levels, I think have DEBUG is fine, but also that
> > means
> > > > without manually turning it on users would not get it. Maybe we should
> > > > consider adding some dynamic setting mechanisms in the future to allow
> > > > users turn it on / off during run-time.
> > > >
> > > >
> > > > Guozhang
> > > >
> > > >
> > > >
> > > > On Tue, May 28, 2019 at 6:23 AM Bruno Cadonna <br...@confluent.io>
> > > wrote:
> > > >
> > > > > Hi,
> > > > >
> > > > > Thank you for your comments.
> > > > >
> > > > > @Bill:
> > > > >
> > > > > 1. It is like Guozhang wrote:
> > > > > - rocksdb-state-id is for key-value stores
> > > > > - rocksdb-session-state-id is for session stores
> > > > > - rocksdb-window-state-id is for window stores
> > > > > These tags are defined in the corresponding store builders and I
> > think
> > > > > it is a good idea to re-use them.
> > > > >
> > > > > 2. I could not find any exposed ticker or histogram to get the total
> > > > > and average number of compactions, although RocksDB dumps the number
> > > > > of compactions between levels in its log files. There is the
> > > > > NUM_SUBCOMPACTIONS_SCHEDULED histogram that gives you statistics
> > about
> > > > > the number of subcompactions actually scheduled during a compaction,
> > > > > but that is not that what you are looking for. If they will expose
> > the
> > > > > number of compaction in the future, we can still add the metrics you
> > > > > propose. I guess, the metric in this KIP that would indirectly be
> > > > > influenced by the number of L0 files would be write-stall-duration.
> > If
> > > > > there are too many compactions this duration should increase.
> > However,
> > > > > this metric is also influenced by memtable flushes.
> > > > >
> > > > > @John:
> > > > >
> > > > > I think it is a good idea to prefix the flush-related metrics with
> > > > > memtable to avoid ambiguity. I changed the KIP accordingly.
> > > > >
> > > > > @Dongjin:
> > > > >
> > > > > For the tag and compaction-related comments, please see my answers to
> > > Bill.
> > > > >
> > > > > I cannot follow your second paragraph. Are you saying that a tuning
> > > > > guide for RocksDB within Streams based on the metrics in this KIP is
> > > > > out of scope? I also think that it doesn't need to be included in
> > this
> > > > > KIP, but it is worth to work on it afterwards.
> > > > >
> > > > > @Guozhang:
> > > > >
> > > > > 1. Thank you for the explanation. I missed that. I modified the KIP
> > > > > accordingly.
> > > > >
> > > > > 2. No, my plan is to record and expose the set of metrics for each
> > > > > RocksDB store separately. Each set of metrics can then be
> > > > > distinguished by its store ID. Do I miss something here?
> > > > >
> > > > > 3. I agree with you and Sophie about user education and that we
> > should
> > > > > work on it after this KIP.
> > > > >
> > > > > 4. I agree also on the user API. However, I would like to open a
> > > > > separate KIP for it because I still need a bit of thinking to get it.
> > > > > I also think it is a good idea to do one step after the other to get
> > > > > at least the built-in RocksDB metrics into the next release.
> > > > > Do you think I chose too many metrics as built-in metrics for this
> > > > > KIP? What do others think?
> > > > >
> > > > > @Sophie:
> > > > >
> > > > > I tend to DEBUG level, but I do not have heart feelings about it. Do
> > > > > you mean to turn it on/off RocksDB metrics in the Streams
> > > > > configuration?
> > > > >
> > > > > Best,
> > > > > Bruno
> > > > >
> > > > > On Tue, May 21, 2019 at 8:02 PM Sophie Blee-Goldman <
> > > sophie@confluent.io>
> > > > > wrote:
> > > > > >
> > > > > > I definitely agree with Guozhang's "meta" comment: if it's possible
> > > to
> > > > > > allow users to pick and choose individual RocksDB metrics that
> > would
> > > be
> > > > > > ideal. One further question is whether these will be debug or info
> > > level
> > > > > > metrics, or a separate level altogether? If there is a nontrivial
> > > > > overhead
> > > > > > associated with attaching RocksDB metrics it would probably be good
> > > to be
> > > > > > able to independently turn on/off Rocks metrics
> > > > > >
> > > > > > On Tue, May 21, 2019 at 9:00 AM Guozhang Wang <wa...@gmail.com>
> > > > > wrote:
> > > > > >
> > > > > > > Hello Bruno,
> > > > > > >
> > > > > > > Thanks for the KIP, I have a few minor comments and a meta one
> > > which
> > > > > are
> > > > > > > relatively aligned with other folks:
> > > > > > >
> > > > > > > Minor:
> > > > > > >
> > > > > > > 1) Regarding the "rocksdb-state-id = [store ID]", to be
> > consistent
> > > with
> > > > > > > other state store metrics (see
> > > > > > >
> > > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-444%3A+Augment+metrics+for+Kafka+Streams
> > > > > ),
> > > > > > > this tag should be either "rocksdb-window-state-id",
> > > > > > > "rocksdb-session-state-id" or "rocksdb-state-id". For window /
> > > session
> > > > > > > store backed by rocksDB, the tags should not be
> > "rocksdb-state-id".
> > > > > > >
> > > > > > > 2) Also for window / session store, my take is that you plan to
> > > have
> > > > > > > multiple rocksDB behind the scene to report to the same set of
> > > > > metrics, is
> > > > > > > that right? My concern is that for such types of state stores,
> > > most of
> > > > > the
> > > > > > > access would be on the first segment rocksDB instance, and hence
> > > > > coalescing
> > > > > > > all of instances as a single set of metrics may dilute it.
> > > > > > >
> > > > > > > 3) I agree with @sophie@confluent.io <so...@confluent.io> that
> > we
> > > > > should
> > > > > > > better have some documentation educating users what to do when
> > see
> > > what
> > > > > > > anomalies in metrics; though I think this is a long endeavoring
> > > effort
> > > > > that
> > > > > > > may go beyond this KIP's scope, let's keep that as a separate
> > > umbrella
> > > > > JIRA
> > > > > > > to accumulate knowledge over time.
> > > > > > >
> > > > > > >
> > > > > > > Meta:
> > > > > > >
> > > > > > > 4) Instead of trying to enumerate all the ones that might be
> > > helpful,
> > > > > I'd
> > > > > > > recommend that we expose a user-friendly API in StreamsMetrics to
> > > allow
> > > > > > > users to add more metrics from RocksDB they'd like to have, while
> > > only
> > > > > > > keeping a small set of most-meaningful ones that are ubiquitously
> > > > > useful
> > > > > > > out-of-the-box. WDYT?
> > > > > > >
> > > > > > >
> > > > > > > Guozhang
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Tue, May 21, 2019 at 8:04 AM Dongjin Lee <do...@apache.org>
> > > > > wrote:
> > > > > > >
> > > > > > >> Hi Bruno,
> > > > > > >>
> > > > > > >> I just read the KIP. I think this feature is great. As far as I
> > > know,
> > > > > most
> > > > > > >> Kafka users monitor the host resources, JVM resources, and Kafka
> > > > > metrics
> > > > > > >> only, not RocksDB for configuring the statistics feature is a
> > > little
> > > > > bit
> > > > > > >> tiresome. Since RocksDB impacts the performance of Kafka
> > Streams,
> > > I
> > > > > > >> believe
> > > > > > >> providing these metrics with other metrics in one place is much
> > > > > better.
> > > > > > >>
> > > > > > >> However, I am a little bit not assured for how much information
> > > > > should be
> > > > > > >> provided to the users with the documentation - how the user can
> > > > > control
> > > > > > >> the
> > > > > > >>  RocksDB may on the boundary of the scope. How do you think?
> > > > > > >>
> > > > > > >> +1. I entirely agree to Bill's comments; I also think
> > > > > `rocksdb-store-id`
> > > > > > >> is
> > > > > > >> better than `rocksdb-state-id` and metrics on total compactions
> > > and an
> > > > > > >> average number of compactions is also needed.
> > > > > > >>
> > > > > > >> Regards,
> > > > > > >> Dongjin
> > > > > > >>
> > > > > > >> On Tue, May 21, 2019 at 2:48 AM John Roesler <john@confluent.io
> > >
> > > > > wrote:
> > > > > > >>
> > > > > > >> > Hi Bruno,
> > > > > > >> >
> > > > > > >> > Looks really good overall. This is going to be an awesome
> > > addition.
> > > > > > >> >
> > > > > > >> > My only thought was that we have "bytes-flushed-(rate|total)
> > and
> > > > > > >> > flush-time-(avg|min|max)" metrics, and the description states
> > > that
> > > > > > >> > these are specifically for Memtable flush operations. What do
> > > you
> > > > > > >> > think about calling it "memtable-bytes-flushed... (etc)"? On
> > one
> > > > > hand,
> > > > > > >> > I could see this being redundant, since that's the only thing
> > > that
> > > > > > >> > gets "flushed" inside of Rocks. But on the other, we have an
> > > > > > >> > independent "flush" operation in streams, which might be
> > > confusing.
> > > > > > >> > Plus, it might help people who are looking at the full "menu"
> > of
> > > > > > >> > hundreds of metrics. They can't read and remember every
> > > description
> > > > > > >> > while trying to understand the full list of metrics, so going
> > > for
> > > > > > >> > maximum self-documentation in the name seems nice.
> > > > > > >> >
> > > > > > >> > But that's a minor thought. Modulo the others' comments, this
> > > looks
> > > > > good
> > > > > > >> > to me.
> > > > > > >> >
> > > > > > >> > Thanks,
> > > > > > >> > -John
> > > > > > >> >
> > > > > > >> > On Mon, May 20, 2019 at 11:07 AM Bill Bejeck <
> > bbejeck@gmail.com
> > > >
> > > > > wrote:
> > > > > > >> > >
> > > > > > >> > > Hi Bruno,
> > > > > > >> > >
> > > > > > >> > > Thanks for the KIP, this will be a useful addition.
> > > > > > >> > >
> > > > > > >> > > Overall the KIP looks good to me, and I have two minor
> > > comments.
> > > > > > >> > >
> > > > > > >> > > 1. For the tags should, I'm wondering if rocksdb-state-id
> > > should
> > > > > be
> > > > > > >> > > rocksdb-store-id
> > > > > > >> > > instead?
> > > > > > >> > >
> > > > > > >> > > 2. With the compaction metrics, would it be possible to add
> > > total
> > > > > > >> > > compactions and an average number of compactions?  I've
> > taken
> > > a
> > > > > look
> > > > > > >> at
> > > > > > >> > the
> > > > > > >> > > available RocksDB metrics, and I'm not sure.  But users can
> > > > > control
> > > > > > >> how
> > > > > > >> > > many L0 files it takes to trigger compaction so if it is
> > > > > possible; it
> > > > > > >> may
> > > > > > >> > > be useful.
> > > > > > >> > >
> > > > > > >> > > Thanks,
> > > > > > >> > > Bill
> > > > > > >> > >
> > > > > > >> > >
> > > > > > >> > > On Mon, May 20, 2019 at 9:15 AM Bruno Cadonna <
> > > bruno@confluent.io
> > > > > >
> > > > > > >> > wrote:
> > > > > > >> > >
> > > > > > >> > > > Hi Sophie,
> > > > > > >> > > >
> > > > > > >> > > > Thank you for your comments.
> > > > > > >> > > >
> > > > > > >> > > > It's a good idea to supplement the metrics with
> > > configuration
> > > > > option
> > > > > > >> > > > to change the metrics. I also had some thoughts about it.
> > > > > However, I
> > > > > > >> > > > think I need some experimentation to get this right.
> > > > > > >> > > >
> > > > > > >> > > > I added the block cache hit rates for index and filter
> > > blocks
> > > > > to the
> > > > > > >> > > > KIP. As far as I understood, they should stay at zero, if
> > > users
> > > > > do
> > > > > > >> not
> > > > > > >> > > > configure RocksDB to include index and filter blocks into
> > > the
> > > > > block
> > > > > > >> > > > cache. Did you also understand this similarly? I guess
> > also
> > > in
> > > > > this
> > > > > > >> > > > case some experimentation would be good to be sure.
> > > > > > >> > > >
> > > > > > >> > > > Best,
> > > > > > >> > > > Bruno
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > > On Sat, May 18, 2019 at 2:29 AM Sophie Blee-Goldman <
> > > > > > >> > sophie@confluent.io>
> > > > > > >> > > > wrote:
> > > > > > >> > > > >
> > > > > > >> > > > > Actually I wonder if it might be useful to users to be
> > > able to
> > > > > > >> break
> > > > > > >> > up
> > > > > > >> > > > the
> > > > > > >> > > > > cache hit stats by type? Some people may choose to store
> > > > > index and
> > > > > > >> > filter
> > > > > > >> > > > > blocks alongside data blocks, and it would probably be
> > > very
> > > > > > >> helpful
> > > > > > >> > for
> > > > > > >> > > > > them to know who is making more effective use of the
> > > cache in
> > > > > > >> order
> > > > > > >> > to
> > > > > > >> > > > tune
> > > > > > >> > > > > how much of it is allocated to each. I'm not sure how
> > > common
> > > > > this
> > > > > > >> > really
> > > > > > >> > > > is
> > > > > > >> > > > > but I think it would be invaluable to those who do.
> > > RocksDB
> > > > > > >> > performance
> > > > > > >> > > > can
> > > > > > >> > > > > be quite opaque..
> > > > > > >> > > > >
> > > > > > >> > > > > Cheers,
> > > > > > >> > > > > Sophie
> > > > > > >> > > > >
> > > > > > >> > > > > On Fri, May 17, 2019 at 5:01 PM Sophie Blee-Goldman <
> > > > > > >> > sophie@confluent.io
> > > > > > >> > > > >
> > > > > > >> > > > > wrote:
> > > > > > >> > > > >
> > > > > > >> > > > > > Hey Bruno!
> > > > > > >> > > > > >
> > > > > > >> > > > > > This all looks pretty good to me, but one suggestion I
> > > have
> > > > > is
> > > > > > >> to
> > > > > > >> > > > > > supplement each of the metrics with some info on how
> > the
> > > > > user
> > > > > > >> can
> > > > > > >> > > > control
> > > > > > >> > > > > > them. In other words, which options could/should they
> > > set in
> > > > > > >> > > > > > RocksDBConfigSetter should they discover a particular
> > > > > > >> bottleneck?
> > > > > > >> > > > > >
> > > > > > >> > > > > > I don't think this necessarily needs to go into the
> > KIP,
> > > > > but I
> > > > > > >> do
> > > > > > >> > > > think it
> > > > > > >> > > > > > should be included in the docs somewhere (happy to
> > help
> > > > > build up
> > > > > > >> > the
> > > > > > >> > > > list
> > > > > > >> > > > > > of associated options when the time comes)
> > > > > > >> > > > > >
> > > > > > >> > > > > > Thanks!
> > > > > > >> > > > > > Sophie
> > > > > > >> > > > > >
> > > > > > >> > > > > > On Fri, May 17, 2019 at 2:54 PM Bruno Cadonna <
> > > > > > >> bruno@confluent.io>
> > > > > > >> > > > wrote:
> > > > > > >> > > > > >
> > > > > > >> > > > > >> Hi all,
> > > > > > >> > > > > >>
> > > > > > >> > > > > >> this KIP describes the extension of the Kafka
> > Streams'
> > > > > metrics
> > > > > > >> to
> > > > > > >> > > > include
> > > > > > >> > > > > >> RocksDB's internal statistics.
> > > > > > >> > > > > >>
> > > > > > >> > > > > >> Please have a look at it and let me know what you
> > > think.
> > > > > Since
> > > > > > >> I
> > > > > > >> > am
> > > > > > >> > > > not a
> > > > > > >> > > > > >> RocksDB expert, I am thankful for any additional pair
> > > of
> > > > > eyes
> > > > > > >> that
> > > > > > >> > > > > >> evaluates this KIP.
> > > > > > >> > > > > >>
> > > > > > >> > > > > >>
> > > > > > >> > > > > >>
> > > > > > >> > > >
> > > > > > >> >
> > > > > > >>
> > > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-471:+Expose+RocksDB+Metrics+in+Kafka+Streams
> > > > > > >> > > > > >>
> > > > > > >> > > > > >> Best regards,
> > > > > > >> > > > > >> Bruno
> > > > > > >> > > > > >>
> > > > > > >> > > > > >
> > > > > > >> > > >
> > > > > > >> >
> > > > > > >>
> > > > > > >>
> > > > > > >> --
> > > > > > >> *Dongjin Lee*
> > > > > > >>
> > > > > > >> *A hitchhiker in the mathematical world.*
> > > > > > >> *github:  <http://goog_969573159/>github.com/dongjinleekr
> > > > > > >> <https://github.com/dongjinleekr>linkedin:
> > > > > > >> kr.linkedin.com/in/dongjinleekr
> > > > > > >> <https://kr.linkedin.com/in/dongjinleekr>speakerdeck:
> > > > > > >> speakerdeck.com/dongjin
> > > > > > >> <https://speakerdeck.com/dongjin>*
> > > > > > >>
> > > > > > >
> > > > > > >
> > > > > > > --
> > > > > > > -- Guozhang
> > > > > > >
> > > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > >
> >
>
>
> --
> -- Guozhang

Re: [DISCUSS] KIP-471: Expose RocksDB Metrics in Kafka Streams

Posted by Guozhang Wang <wa...@gmail.com>.
I think Bruno's 2) is that for a segmented store, the access rate on
different segments will very likely be different. And in fact, most of the
access should be on the "latest" segment unless 1) very late arrived data,
which should be captured on the higher-level `lateness` metrics already,
and 2) IQ reads on old windows. The problem is that, say if 99% of reads go
to the latest segment, and 1% goes to rest of the segments, how should
`memtable-hit-rate` be calculated then.

Another wild thought just to throw here: maybe we can just expose the
latest segment's state store as the logical store's metrics? Admittedly it
would not be most accurate, but it is 1) future-proof if we want to
consolidate to 1-1 physical store -> logical store implementation, and 2)
it is as simple and not needing to bookkeep older segments who should be
rarely accessed. My question is though, if upon segment rolling our metrics
can be auto-switched to the new store.


Guozhang

On Tue, Jun 4, 2019 at 3:06 PM Sophie Blee-Goldman <so...@confluent.io>
wrote:

> Hey Bruno,
>
> I tend to agree with Guozhang on this matter although you do bring up some
> good points that should be addressed. Regarding 1) I think it is probably
> fairly uncommon in practice for users to leverage the individual store
> names passed to RocksDBConfigSetter#setConfig in order to specify options
> on a per-store basis. When this actually is used, it does seem likely that
> users would be doing something like pattern matching the physical store
> name prefix in order to apply configs to all physical stores (segments)
> within a single logical RocksDBStore. As you mention this is something of a
> hassle already as physical stores are created/deleted, while most likely
> all anyone cares about is the prefix corresponding to the logical store. It
> seems like rather than persist this hassle to the metric layer, we should
> consider refactoring RocksDBConfigSetter to apply to a logical store rather
> than a specific physical segment. Or maybe providing some kind of tooling
> to at least make this easier on users, but that's definitely outside the
> scope of this KIP.
>
> Regarding 2) can you clarify your point about accessing stores uniformly?
> While I agree there will definitely be variance in the access pattern of
> different segments, I think it's unlikely that it will vary in any kind of
> predictable or deterministic way, hence it is not that useful to know in
> hindsight the difference reflected by the metrics.
>
> Cheers,
> Sophie
>
> On Tue, Jun 4, 2019 at 2:09 PM Bruno Cadonna <br...@confluent.io> wrote:
>
> > Hi Guozhang,
> >
> > After some thoughts, I tend to be in favour of the option with metrics
> > for each physical RocksDB instance for the following reasons:
> >
> > 1) A user already needs to be aware of segmented state stores when
> > providing a custom RocksDBConfigSetter. In RocksDBConfigSetter one can
> > specify settings for a store depending on the name of the store. Since
> > segments (i.e. state store) in a segmented state store have names that
> > share a prefix but have suffixes that are created at runtime, increase
> > with time and are theoretically unbounded, a user needs to take
> > account of the segments to provide the settings for all (i.e. matching
> > the common prefix) or some (i.e. matching the common prefix and for
> > example suffixes according to a specific pattern) of the segments of a
> > specific segmented state store.
> > 2) Currently settings for RocksDB can only be specified by a user per
> > physical instance and not per logical instance. Deriving good settings
> > for physical instances from metrics for a logical instance can be hard
> > if the physical instances are not accessed uniformly. In segmented
> > state stores segments are not accessed uniformly.
> > 3) Simpler to implement and to get things done.
> >
> > Any thoughts on this from anybody?
> >
> > Best,
> > Bruno
> >
> > On Thu, May 30, 2019 at 8:33 PM Guozhang Wang <wa...@gmail.com>
> wrote:
> > >
> > > Hi Bruno:
> > >
> > > Regarding 2) I think either way has some shortcomings: exposing the
> > metrics
> > > per rocksDB instance for window / session stores exposed some
> > > implementation internals (that we use segmented stores) to enforce
> users
> > to
> > > be aware of them. E.g. what if we want to silently change the internal
> > > implementation by walking away from the segmented approach? On the
> other
> > > hand, coalescing multiple rocksDB instances' metrics into a single one
> > per
> > > each logical store also has some concerns as I mentioned above. What
> I'm
> > > thinking is actually that, if we can customize the aggregation logic to
> > > still has one set of metrics per each logical store which may be
> composed
> > > of multiple rocksDB ones -- e.g. for `bytes-written-rate` we sum them
> > > across rocksDBs, while for `memtable-hit-rate` we do weighted average?
> > >
> > > Regarding logging levels, I think have DEBUG is fine, but also that
> means
> > > without manually turning it on users would not get it. Maybe we should
> > > consider adding some dynamic setting mechanisms in the future to allow
> > > users turn it on / off during run-time.
> > >
> > >
> > > Guozhang
> > >
> > >
> > >
> > > On Tue, May 28, 2019 at 6:23 AM Bruno Cadonna <br...@confluent.io>
> > wrote:
> > >
> > > > Hi,
> > > >
> > > > Thank you for your comments.
> > > >
> > > > @Bill:
> > > >
> > > > 1. It is like Guozhang wrote:
> > > > - rocksdb-state-id is for key-value stores
> > > > - rocksdb-session-state-id is for session stores
> > > > - rocksdb-window-state-id is for window stores
> > > > These tags are defined in the corresponding store builders and I
> think
> > > > it is a good idea to re-use them.
> > > >
> > > > 2. I could not find any exposed ticker or histogram to get the total
> > > > and average number of compactions, although RocksDB dumps the number
> > > > of compactions between levels in its log files. There is the
> > > > NUM_SUBCOMPACTIONS_SCHEDULED histogram that gives you statistics
> about
> > > > the number of subcompactions actually scheduled during a compaction,
> > > > but that is not that what you are looking for. If they will expose
> the
> > > > number of compaction in the future, we can still add the metrics you
> > > > propose. I guess, the metric in this KIP that would indirectly be
> > > > influenced by the number of L0 files would be write-stall-duration.
> If
> > > > there are too many compactions this duration should increase.
> However,
> > > > this metric is also influenced by memtable flushes.
> > > >
> > > > @John:
> > > >
> > > > I think it is a good idea to prefix the flush-related metrics with
> > > > memtable to avoid ambiguity. I changed the KIP accordingly.
> > > >
> > > > @Dongjin:
> > > >
> > > > For the tag and compaction-related comments, please see my answers to
> > Bill.
> > > >
> > > > I cannot follow your second paragraph. Are you saying that a tuning
> > > > guide for RocksDB within Streams based on the metrics in this KIP is
> > > > out of scope? I also think that it doesn't need to be included in
> this
> > > > KIP, but it is worth to work on it afterwards.
> > > >
> > > > @Guozhang:
> > > >
> > > > 1. Thank you for the explanation. I missed that. I modified the KIP
> > > > accordingly.
> > > >
> > > > 2. No, my plan is to record and expose the set of metrics for each
> > > > RocksDB store separately. Each set of metrics can then be
> > > > distinguished by its store ID. Do I miss something here?
> > > >
> > > > 3. I agree with you and Sophie about user education and that we
> should
> > > > work on it after this KIP.
> > > >
> > > > 4. I agree also on the user API. However, I would like to open a
> > > > separate KIP for it because I still need a bit of thinking to get it.
> > > > I also think it is a good idea to do one step after the other to get
> > > > at least the built-in RocksDB metrics into the next release.
> > > > Do you think I chose too many metrics as built-in metrics for this
> > > > KIP? What do others think?
> > > >
> > > > @Sophie:
> > > >
> > > > I tend to DEBUG level, but I do not have heart feelings about it. Do
> > > > you mean to turn it on/off RocksDB metrics in the Streams
> > > > configuration?
> > > >
> > > > Best,
> > > > Bruno
> > > >
> > > > On Tue, May 21, 2019 at 8:02 PM Sophie Blee-Goldman <
> > sophie@confluent.io>
> > > > wrote:
> > > > >
> > > > > I definitely agree with Guozhang's "meta" comment: if it's possible
> > to
> > > > > allow users to pick and choose individual RocksDB metrics that
> would
> > be
> > > > > ideal. One further question is whether these will be debug or info
> > level
> > > > > metrics, or a separate level altogether? If there is a nontrivial
> > > > overhead
> > > > > associated with attaching RocksDB metrics it would probably be good
> > to be
> > > > > able to independently turn on/off Rocks metrics
> > > > >
> > > > > On Tue, May 21, 2019 at 9:00 AM Guozhang Wang <wa...@gmail.com>
> > > > wrote:
> > > > >
> > > > > > Hello Bruno,
> > > > > >
> > > > > > Thanks for the KIP, I have a few minor comments and a meta one
> > which
> > > > are
> > > > > > relatively aligned with other folks:
> > > > > >
> > > > > > Minor:
> > > > > >
> > > > > > 1) Regarding the "rocksdb-state-id = [store ID]", to be
> consistent
> > with
> > > > > > other state store metrics (see
> > > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-444%3A+Augment+metrics+for+Kafka+Streams
> > > > ),
> > > > > > this tag should be either "rocksdb-window-state-id",
> > > > > > "rocksdb-session-state-id" or "rocksdb-state-id". For window /
> > session
> > > > > > store backed by rocksDB, the tags should not be
> "rocksdb-state-id".
> > > > > >
> > > > > > 2) Also for window / session store, my take is that you plan to
> > have
> > > > > > multiple rocksDB behind the scene to report to the same set of
> > > > metrics, is
> > > > > > that right? My concern is that for such types of state stores,
> > most of
> > > > the
> > > > > > access would be on the first segment rocksDB instance, and hence
> > > > coalescing
> > > > > > all of instances as a single set of metrics may dilute it.
> > > > > >
> > > > > > 3) I agree with @sophie@confluent.io <so...@confluent.io> that
> we
> > > > should
> > > > > > better have some documentation educating users what to do when
> see
> > what
> > > > > > anomalies in metrics; though I think this is a long endeavoring
> > effort
> > > > that
> > > > > > may go beyond this KIP's scope, let's keep that as a separate
> > umbrella
> > > > JIRA
> > > > > > to accumulate knowledge over time.
> > > > > >
> > > > > >
> > > > > > Meta:
> > > > > >
> > > > > > 4) Instead of trying to enumerate all the ones that might be
> > helpful,
> > > > I'd
> > > > > > recommend that we expose a user-friendly API in StreamsMetrics to
> > allow
> > > > > > users to add more metrics from RocksDB they'd like to have, while
> > only
> > > > > > keeping a small set of most-meaningful ones that are ubiquitously
> > > > useful
> > > > > > out-of-the-box. WDYT?
> > > > > >
> > > > > >
> > > > > > Guozhang
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Tue, May 21, 2019 at 8:04 AM Dongjin Lee <do...@apache.org>
> > > > wrote:
> > > > > >
> > > > > >> Hi Bruno,
> > > > > >>
> > > > > >> I just read the KIP. I think this feature is great. As far as I
> > know,
> > > > most
> > > > > >> Kafka users monitor the host resources, JVM resources, and Kafka
> > > > metrics
> > > > > >> only, not RocksDB for configuring the statistics feature is a
> > little
> > > > bit
> > > > > >> tiresome. Since RocksDB impacts the performance of Kafka
> Streams,
> > I
> > > > > >> believe
> > > > > >> providing these metrics with other metrics in one place is much
> > > > better.
> > > > > >>
> > > > > >> However, I am a little bit not assured for how much information
> > > > should be
> > > > > >> provided to the users with the documentation - how the user can
> > > > control
> > > > > >> the
> > > > > >>  RocksDB may on the boundary of the scope. How do you think?
> > > > > >>
> > > > > >> +1. I entirely agree to Bill's comments; I also think
> > > > `rocksdb-store-id`
> > > > > >> is
> > > > > >> better than `rocksdb-state-id` and metrics on total compactions
> > and an
> > > > > >> average number of compactions is also needed.
> > > > > >>
> > > > > >> Regards,
> > > > > >> Dongjin
> > > > > >>
> > > > > >> On Tue, May 21, 2019 at 2:48 AM John Roesler <john@confluent.io
> >
> > > > wrote:
> > > > > >>
> > > > > >> > Hi Bruno,
> > > > > >> >
> > > > > >> > Looks really good overall. This is going to be an awesome
> > addition.
> > > > > >> >
> > > > > >> > My only thought was that we have "bytes-flushed-(rate|total)
> and
> > > > > >> > flush-time-(avg|min|max)" metrics, and the description states
> > that
> > > > > >> > these are specifically for Memtable flush operations. What do
> > you
> > > > > >> > think about calling it "memtable-bytes-flushed... (etc)"? On
> one
> > > > hand,
> > > > > >> > I could see this being redundant, since that's the only thing
> > that
> > > > > >> > gets "flushed" inside of Rocks. But on the other, we have an
> > > > > >> > independent "flush" operation in streams, which might be
> > confusing.
> > > > > >> > Plus, it might help people who are looking at the full "menu"
> of
> > > > > >> > hundreds of metrics. They can't read and remember every
> > description
> > > > > >> > while trying to understand the full list of metrics, so going
> > for
> > > > > >> > maximum self-documentation in the name seems nice.
> > > > > >> >
> > > > > >> > But that's a minor thought. Modulo the others' comments, this
> > looks
> > > > good
> > > > > >> > to me.
> > > > > >> >
> > > > > >> > Thanks,
> > > > > >> > -John
> > > > > >> >
> > > > > >> > On Mon, May 20, 2019 at 11:07 AM Bill Bejeck <
> bbejeck@gmail.com
> > >
> > > > wrote:
> > > > > >> > >
> > > > > >> > > Hi Bruno,
> > > > > >> > >
> > > > > >> > > Thanks for the KIP, this will be a useful addition.
> > > > > >> > >
> > > > > >> > > Overall the KIP looks good to me, and I have two minor
> > comments.
> > > > > >> > >
> > > > > >> > > 1. For the tags should, I'm wondering if rocksdb-state-id
> > should
> > > > be
> > > > > >> > > rocksdb-store-id
> > > > > >> > > instead?
> > > > > >> > >
> > > > > >> > > 2. With the compaction metrics, would it be possible to add
> > total
> > > > > >> > > compactions and an average number of compactions?  I've
> taken
> > a
> > > > look
> > > > > >> at
> > > > > >> > the
> > > > > >> > > available RocksDB metrics, and I'm not sure.  But users can
> > > > control
> > > > > >> how
> > > > > >> > > many L0 files it takes to trigger compaction so if it is
> > > > possible; it
> > > > > >> may
> > > > > >> > > be useful.
> > > > > >> > >
> > > > > >> > > Thanks,
> > > > > >> > > Bill
> > > > > >> > >
> > > > > >> > >
> > > > > >> > > On Mon, May 20, 2019 at 9:15 AM Bruno Cadonna <
> > bruno@confluent.io
> > > > >
> > > > > >> > wrote:
> > > > > >> > >
> > > > > >> > > > Hi Sophie,
> > > > > >> > > >
> > > > > >> > > > Thank you for your comments.
> > > > > >> > > >
> > > > > >> > > > It's a good idea to supplement the metrics with
> > configuration
> > > > option
> > > > > >> > > > to change the metrics. I also had some thoughts about it.
> > > > However, I
> > > > > >> > > > think I need some experimentation to get this right.
> > > > > >> > > >
> > > > > >> > > > I added the block cache hit rates for index and filter
> > blocks
> > > > to the
> > > > > >> > > > KIP. As far as I understood, they should stay at zero, if
> > users
> > > > do
> > > > > >> not
> > > > > >> > > > configure RocksDB to include index and filter blocks into
> > the
> > > > block
> > > > > >> > > > cache. Did you also understand this similarly? I guess
> also
> > in
> > > > this
> > > > > >> > > > case some experimentation would be good to be sure.
> > > > > >> > > >
> > > > > >> > > > Best,
> > > > > >> > > > Bruno
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > > On Sat, May 18, 2019 at 2:29 AM Sophie Blee-Goldman <
> > > > > >> > sophie@confluent.io>
> > > > > >> > > > wrote:
> > > > > >> > > > >
> > > > > >> > > > > Actually I wonder if it might be useful to users to be
> > able to
> > > > > >> break
> > > > > >> > up
> > > > > >> > > > the
> > > > > >> > > > > cache hit stats by type? Some people may choose to store
> > > > index and
> > > > > >> > filter
> > > > > >> > > > > blocks alongside data blocks, and it would probably be
> > very
> > > > > >> helpful
> > > > > >> > for
> > > > > >> > > > > them to know who is making more effective use of the
> > cache in
> > > > > >> order
> > > > > >> > to
> > > > > >> > > > tune
> > > > > >> > > > > how much of it is allocated to each. I'm not sure how
> > common
> > > > this
> > > > > >> > really
> > > > > >> > > > is
> > > > > >> > > > > but I think it would be invaluable to those who do.
> > RocksDB
> > > > > >> > performance
> > > > > >> > > > can
> > > > > >> > > > > be quite opaque..
> > > > > >> > > > >
> > > > > >> > > > > Cheers,
> > > > > >> > > > > Sophie
> > > > > >> > > > >
> > > > > >> > > > > On Fri, May 17, 2019 at 5:01 PM Sophie Blee-Goldman <
> > > > > >> > sophie@confluent.io
> > > > > >> > > > >
> > > > > >> > > > > wrote:
> > > > > >> > > > >
> > > > > >> > > > > > Hey Bruno!
> > > > > >> > > > > >
> > > > > >> > > > > > This all looks pretty good to me, but one suggestion I
> > have
> > > > is
> > > > > >> to
> > > > > >> > > > > > supplement each of the metrics with some info on how
> the
> > > > user
> > > > > >> can
> > > > > >> > > > control
> > > > > >> > > > > > them. In other words, which options could/should they
> > set in
> > > > > >> > > > > > RocksDBConfigSetter should they discover a particular
> > > > > >> bottleneck?
> > > > > >> > > > > >
> > > > > >> > > > > > I don't think this necessarily needs to go into the
> KIP,
> > > > but I
> > > > > >> do
> > > > > >> > > > think it
> > > > > >> > > > > > should be included in the docs somewhere (happy to
> help
> > > > build up
> > > > > >> > the
> > > > > >> > > > list
> > > > > >> > > > > > of associated options when the time comes)
> > > > > >> > > > > >
> > > > > >> > > > > > Thanks!
> > > > > >> > > > > > Sophie
> > > > > >> > > > > >
> > > > > >> > > > > > On Fri, May 17, 2019 at 2:54 PM Bruno Cadonna <
> > > > > >> bruno@confluent.io>
> > > > > >> > > > wrote:
> > > > > >> > > > > >
> > > > > >> > > > > >> Hi all,
> > > > > >> > > > > >>
> > > > > >> > > > > >> this KIP describes the extension of the Kafka
> Streams'
> > > > metrics
> > > > > >> to
> > > > > >> > > > include
> > > > > >> > > > > >> RocksDB's internal statistics.
> > > > > >> > > > > >>
> > > > > >> > > > > >> Please have a look at it and let me know what you
> > think.
> > > > Since
> > > > > >> I
> > > > > >> > am
> > > > > >> > > > not a
> > > > > >> > > > > >> RocksDB expert, I am thankful for any additional pair
> > of
> > > > eyes
> > > > > >> that
> > > > > >> > > > > >> evaluates this KIP.
> > > > > >> > > > > >>
> > > > > >> > > > > >>
> > > > > >> > > > > >>
> > > > > >> > > >
> > > > > >> >
> > > > > >>
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-471:+Expose+RocksDB+Metrics+in+Kafka+Streams
> > > > > >> > > > > >>
> > > > > >> > > > > >> Best regards,
> > > > > >> > > > > >> Bruno
> > > > > >> > > > > >>
> > > > > >> > > > > >
> > > > > >> > > >
> > > > > >> >
> > > > > >>
> > > > > >>
> > > > > >> --
> > > > > >> *Dongjin Lee*
> > > > > >>
> > > > > >> *A hitchhiker in the mathematical world.*
> > > > > >> *github:  <http://goog_969573159/>github.com/dongjinleekr
> > > > > >> <https://github.com/dongjinleekr>linkedin:
> > > > > >> kr.linkedin.com/in/dongjinleekr
> > > > > >> <https://kr.linkedin.com/in/dongjinleekr>speakerdeck:
> > > > > >> speakerdeck.com/dongjin
> > > > > >> <https://speakerdeck.com/dongjin>*
> > > > > >>
> > > > > >
> > > > > >
> > > > > > --
> > > > > > -- Guozhang
> > > > > >
> > > >
> > >
> > >
> > > --
> > > -- Guozhang
> >
>


-- 
-- Guozhang

Re: [DISCUSS] KIP-471: Expose RocksDB Metrics in Kafka Streams

Posted by Sophie Blee-Goldman <so...@confluent.io>.
Hey Bruno,

I tend to agree with Guozhang on this matter although you do bring up some
good points that should be addressed. Regarding 1) I think it is probably
fairly uncommon in practice for users to leverage the individual store
names passed to RocksDBConfigSetter#setConfig in order to specify options
on a per-store basis. When this actually is used, it does seem likely that
users would be doing something like pattern matching the physical store
name prefix in order to apply configs to all physical stores (segments)
within a single logical RocksDBStore. As you mention this is something of a
hassle already as physical stores are created/deleted, while most likely
all anyone cares about is the prefix corresponding to the logical store. It
seems like rather than persist this hassle to the metric layer, we should
consider refactoring RocksDBConfigSetter to apply to a logical store rather
than a specific physical segment. Or maybe providing some kind of tooling
to at least make this easier on users, but that's definitely outside the
scope of this KIP.

Regarding 2) can you clarify your point about accessing stores uniformly?
While I agree there will definitely be variance in the access pattern of
different segments, I think it's unlikely that it will vary in any kind of
predictable or deterministic way, hence it is not that useful to know in
hindsight the difference reflected by the metrics.

Cheers,
Sophie

On Tue, Jun 4, 2019 at 2:09 PM Bruno Cadonna <br...@confluent.io> wrote:

> Hi Guozhang,
>
> After some thoughts, I tend to be in favour of the option with metrics
> for each physical RocksDB instance for the following reasons:
>
> 1) A user already needs to be aware of segmented state stores when
> providing a custom RocksDBConfigSetter. In RocksDBConfigSetter one can
> specify settings for a store depending on the name of the store. Since
> segments (i.e. state store) in a segmented state store have names that
> share a prefix but have suffixes that are created at runtime, increase
> with time and are theoretically unbounded, a user needs to take
> account of the segments to provide the settings for all (i.e. matching
> the common prefix) or some (i.e. matching the common prefix and for
> example suffixes according to a specific pattern) of the segments of a
> specific segmented state store.
> 2) Currently settings for RocksDB can only be specified by a user per
> physical instance and not per logical instance. Deriving good settings
> for physical instances from metrics for a logical instance can be hard
> if the physical instances are not accessed uniformly. In segmented
> state stores segments are not accessed uniformly.
> 3) Simpler to implement and to get things done.
>
> Any thoughts on this from anybody?
>
> Best,
> Bruno
>
> On Thu, May 30, 2019 at 8:33 PM Guozhang Wang <wa...@gmail.com> wrote:
> >
> > Hi Bruno:
> >
> > Regarding 2) I think either way has some shortcomings: exposing the
> metrics
> > per rocksDB instance for window / session stores exposed some
> > implementation internals (that we use segmented stores) to enforce users
> to
> > be aware of them. E.g. what if we want to silently change the internal
> > implementation by walking away from the segmented approach? On the other
> > hand, coalescing multiple rocksDB instances' metrics into a single one
> per
> > each logical store also has some concerns as I mentioned above. What I'm
> > thinking is actually that, if we can customize the aggregation logic to
> > still has one set of metrics per each logical store which may be composed
> > of multiple rocksDB ones -- e.g. for `bytes-written-rate` we sum them
> > across rocksDBs, while for `memtable-hit-rate` we do weighted average?
> >
> > Regarding logging levels, I think have DEBUG is fine, but also that means
> > without manually turning it on users would not get it. Maybe we should
> > consider adding some dynamic setting mechanisms in the future to allow
> > users turn it on / off during run-time.
> >
> >
> > Guozhang
> >
> >
> >
> > On Tue, May 28, 2019 at 6:23 AM Bruno Cadonna <br...@confluent.io>
> wrote:
> >
> > > Hi,
> > >
> > > Thank you for your comments.
> > >
> > > @Bill:
> > >
> > > 1. It is like Guozhang wrote:
> > > - rocksdb-state-id is for key-value stores
> > > - rocksdb-session-state-id is for session stores
> > > - rocksdb-window-state-id is for window stores
> > > These tags are defined in the corresponding store builders and I think
> > > it is a good idea to re-use them.
> > >
> > > 2. I could not find any exposed ticker or histogram to get the total
> > > and average number of compactions, although RocksDB dumps the number
> > > of compactions between levels in its log files. There is the
> > > NUM_SUBCOMPACTIONS_SCHEDULED histogram that gives you statistics about
> > > the number of subcompactions actually scheduled during a compaction,
> > > but that is not that what you are looking for. If they will expose the
> > > number of compaction in the future, we can still add the metrics you
> > > propose. I guess, the metric in this KIP that would indirectly be
> > > influenced by the number of L0 files would be write-stall-duration. If
> > > there are too many compactions this duration should increase. However,
> > > this metric is also influenced by memtable flushes.
> > >
> > > @John:
> > >
> > > I think it is a good idea to prefix the flush-related metrics with
> > > memtable to avoid ambiguity. I changed the KIP accordingly.
> > >
> > > @Dongjin:
> > >
> > > For the tag and compaction-related comments, please see my answers to
> Bill.
> > >
> > > I cannot follow your second paragraph. Are you saying that a tuning
> > > guide for RocksDB within Streams based on the metrics in this KIP is
> > > out of scope? I also think that it doesn't need to be included in this
> > > KIP, but it is worth to work on it afterwards.
> > >
> > > @Guozhang:
> > >
> > > 1. Thank you for the explanation. I missed that. I modified the KIP
> > > accordingly.
> > >
> > > 2. No, my plan is to record and expose the set of metrics for each
> > > RocksDB store separately. Each set of metrics can then be
> > > distinguished by its store ID. Do I miss something here?
> > >
> > > 3. I agree with you and Sophie about user education and that we should
> > > work on it after this KIP.
> > >
> > > 4. I agree also on the user API. However, I would like to open a
> > > separate KIP for it because I still need a bit of thinking to get it.
> > > I also think it is a good idea to do one step after the other to get
> > > at least the built-in RocksDB metrics into the next release.
> > > Do you think I chose too many metrics as built-in metrics for this
> > > KIP? What do others think?
> > >
> > > @Sophie:
> > >
> > > I tend to DEBUG level, but I do not have heart feelings about it. Do
> > > you mean to turn it on/off RocksDB metrics in the Streams
> > > configuration?
> > >
> > > Best,
> > > Bruno
> > >
> > > On Tue, May 21, 2019 at 8:02 PM Sophie Blee-Goldman <
> sophie@confluent.io>
> > > wrote:
> > > >
> > > > I definitely agree with Guozhang's "meta" comment: if it's possible
> to
> > > > allow users to pick and choose individual RocksDB metrics that would
> be
> > > > ideal. One further question is whether these will be debug or info
> level
> > > > metrics, or a separate level altogether? If there is a nontrivial
> > > overhead
> > > > associated with attaching RocksDB metrics it would probably be good
> to be
> > > > able to independently turn on/off Rocks metrics
> > > >
> > > > On Tue, May 21, 2019 at 9:00 AM Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >
> > > > > Hello Bruno,
> > > > >
> > > > > Thanks for the KIP, I have a few minor comments and a meta one
> which
> > > are
> > > > > relatively aligned with other folks:
> > > > >
> > > > > Minor:
> > > > >
> > > > > 1) Regarding the "rocksdb-state-id = [store ID]", to be consistent
> with
> > > > > other state store metrics (see
> > > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-444%3A+Augment+metrics+for+Kafka+Streams
> > > ),
> > > > > this tag should be either "rocksdb-window-state-id",
> > > > > "rocksdb-session-state-id" or "rocksdb-state-id". For window /
> session
> > > > > store backed by rocksDB, the tags should not be "rocksdb-state-id".
> > > > >
> > > > > 2) Also for window / session store, my take is that you plan to
> have
> > > > > multiple rocksDB behind the scene to report to the same set of
> > > metrics, is
> > > > > that right? My concern is that for such types of state stores,
> most of
> > > the
> > > > > access would be on the first segment rocksDB instance, and hence
> > > coalescing
> > > > > all of instances as a single set of metrics may dilute it.
> > > > >
> > > > > 3) I agree with @sophie@confluent.io <so...@confluent.io> that we
> > > should
> > > > > better have some documentation educating users what to do when see
> what
> > > > > anomalies in metrics; though I think this is a long endeavoring
> effort
> > > that
> > > > > may go beyond this KIP's scope, let's keep that as a separate
> umbrella
> > > JIRA
> > > > > to accumulate knowledge over time.
> > > > >
> > > > >
> > > > > Meta:
> > > > >
> > > > > 4) Instead of trying to enumerate all the ones that might be
> helpful,
> > > I'd
> > > > > recommend that we expose a user-friendly API in StreamsMetrics to
> allow
> > > > > users to add more metrics from RocksDB they'd like to have, while
> only
> > > > > keeping a small set of most-meaningful ones that are ubiquitously
> > > useful
> > > > > out-of-the-box. WDYT?
> > > > >
> > > > >
> > > > > Guozhang
> > > > >
> > > > >
> > > > >
> > > > > On Tue, May 21, 2019 at 8:04 AM Dongjin Lee <do...@apache.org>
> > > wrote:
> > > > >
> > > > >> Hi Bruno,
> > > > >>
> > > > >> I just read the KIP. I think this feature is great. As far as I
> know,
> > > most
> > > > >> Kafka users monitor the host resources, JVM resources, and Kafka
> > > metrics
> > > > >> only, not RocksDB for configuring the statistics feature is a
> little
> > > bit
> > > > >> tiresome. Since RocksDB impacts the performance of Kafka Streams,
> I
> > > > >> believe
> > > > >> providing these metrics with other metrics in one place is much
> > > better.
> > > > >>
> > > > >> However, I am a little bit not assured for how much information
> > > should be
> > > > >> provided to the users with the documentation - how the user can
> > > control
> > > > >> the
> > > > >>  RocksDB may on the boundary of the scope. How do you think?
> > > > >>
> > > > >> +1. I entirely agree to Bill's comments; I also think
> > > `rocksdb-store-id`
> > > > >> is
> > > > >> better than `rocksdb-state-id` and metrics on total compactions
> and an
> > > > >> average number of compactions is also needed.
> > > > >>
> > > > >> Regards,
> > > > >> Dongjin
> > > > >>
> > > > >> On Tue, May 21, 2019 at 2:48 AM John Roesler <jo...@confluent.io>
> > > wrote:
> > > > >>
> > > > >> > Hi Bruno,
> > > > >> >
> > > > >> > Looks really good overall. This is going to be an awesome
> addition.
> > > > >> >
> > > > >> > My only thought was that we have "bytes-flushed-(rate|total) and
> > > > >> > flush-time-(avg|min|max)" metrics, and the description states
> that
> > > > >> > these are specifically for Memtable flush operations. What do
> you
> > > > >> > think about calling it "memtable-bytes-flushed... (etc)"? On one
> > > hand,
> > > > >> > I could see this being redundant, since that's the only thing
> that
> > > > >> > gets "flushed" inside of Rocks. But on the other, we have an
> > > > >> > independent "flush" operation in streams, which might be
> confusing.
> > > > >> > Plus, it might help people who are looking at the full "menu" of
> > > > >> > hundreds of metrics. They can't read and remember every
> description
> > > > >> > while trying to understand the full list of metrics, so going
> for
> > > > >> > maximum self-documentation in the name seems nice.
> > > > >> >
> > > > >> > But that's a minor thought. Modulo the others' comments, this
> looks
> > > good
> > > > >> > to me.
> > > > >> >
> > > > >> > Thanks,
> > > > >> > -John
> > > > >> >
> > > > >> > On Mon, May 20, 2019 at 11:07 AM Bill Bejeck <bbejeck@gmail.com
> >
> > > wrote:
> > > > >> > >
> > > > >> > > Hi Bruno,
> > > > >> > >
> > > > >> > > Thanks for the KIP, this will be a useful addition.
> > > > >> > >
> > > > >> > > Overall the KIP looks good to me, and I have two minor
> comments.
> > > > >> > >
> > > > >> > > 1. For the tags should, I'm wondering if rocksdb-state-id
> should
> > > be
> > > > >> > > rocksdb-store-id
> > > > >> > > instead?
> > > > >> > >
> > > > >> > > 2. With the compaction metrics, would it be possible to add
> total
> > > > >> > > compactions and an average number of compactions?  I've taken
> a
> > > look
> > > > >> at
> > > > >> > the
> > > > >> > > available RocksDB metrics, and I'm not sure.  But users can
> > > control
> > > > >> how
> > > > >> > > many L0 files it takes to trigger compaction so if it is
> > > possible; it
> > > > >> may
> > > > >> > > be useful.
> > > > >> > >
> > > > >> > > Thanks,
> > > > >> > > Bill
> > > > >> > >
> > > > >> > >
> > > > >> > > On Mon, May 20, 2019 at 9:15 AM Bruno Cadonna <
> bruno@confluent.io
> > > >
> > > > >> > wrote:
> > > > >> > >
> > > > >> > > > Hi Sophie,
> > > > >> > > >
> > > > >> > > > Thank you for your comments.
> > > > >> > > >
> > > > >> > > > It's a good idea to supplement the metrics with
> configuration
> > > option
> > > > >> > > > to change the metrics. I also had some thoughts about it.
> > > However, I
> > > > >> > > > think I need some experimentation to get this right.
> > > > >> > > >
> > > > >> > > > I added the block cache hit rates for index and filter
> blocks
> > > to the
> > > > >> > > > KIP. As far as I understood, they should stay at zero, if
> users
> > > do
> > > > >> not
> > > > >> > > > configure RocksDB to include index and filter blocks into
> the
> > > block
> > > > >> > > > cache. Did you also understand this similarly? I guess also
> in
> > > this
> > > > >> > > > case some experimentation would be good to be sure.
> > > > >> > > >
> > > > >> > > > Best,
> > > > >> > > > Bruno
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > On Sat, May 18, 2019 at 2:29 AM Sophie Blee-Goldman <
> > > > >> > sophie@confluent.io>
> > > > >> > > > wrote:
> > > > >> > > > >
> > > > >> > > > > Actually I wonder if it might be useful to users to be
> able to
> > > > >> break
> > > > >> > up
> > > > >> > > > the
> > > > >> > > > > cache hit stats by type? Some people may choose to store
> > > index and
> > > > >> > filter
> > > > >> > > > > blocks alongside data blocks, and it would probably be
> very
> > > > >> helpful
> > > > >> > for
> > > > >> > > > > them to know who is making more effective use of the
> cache in
> > > > >> order
> > > > >> > to
> > > > >> > > > tune
> > > > >> > > > > how much of it is allocated to each. I'm not sure how
> common
> > > this
> > > > >> > really
> > > > >> > > > is
> > > > >> > > > > but I think it would be invaluable to those who do.
> RocksDB
> > > > >> > performance
> > > > >> > > > can
> > > > >> > > > > be quite opaque..
> > > > >> > > > >
> > > > >> > > > > Cheers,
> > > > >> > > > > Sophie
> > > > >> > > > >
> > > > >> > > > > On Fri, May 17, 2019 at 5:01 PM Sophie Blee-Goldman <
> > > > >> > sophie@confluent.io
> > > > >> > > > >
> > > > >> > > > > wrote:
> > > > >> > > > >
> > > > >> > > > > > Hey Bruno!
> > > > >> > > > > >
> > > > >> > > > > > This all looks pretty good to me, but one suggestion I
> have
> > > is
> > > > >> to
> > > > >> > > > > > supplement each of the metrics with some info on how the
> > > user
> > > > >> can
> > > > >> > > > control
> > > > >> > > > > > them. In other words, which options could/should they
> set in
> > > > >> > > > > > RocksDBConfigSetter should they discover a particular
> > > > >> bottleneck?
> > > > >> > > > > >
> > > > >> > > > > > I don't think this necessarily needs to go into the KIP,
> > > but I
> > > > >> do
> > > > >> > > > think it
> > > > >> > > > > > should be included in the docs somewhere (happy to help
> > > build up
> > > > >> > the
> > > > >> > > > list
> > > > >> > > > > > of associated options when the time comes)
> > > > >> > > > > >
> > > > >> > > > > > Thanks!
> > > > >> > > > > > Sophie
> > > > >> > > > > >
> > > > >> > > > > > On Fri, May 17, 2019 at 2:54 PM Bruno Cadonna <
> > > > >> bruno@confluent.io>
> > > > >> > > > wrote:
> > > > >> > > > > >
> > > > >> > > > > >> Hi all,
> > > > >> > > > > >>
> > > > >> > > > > >> this KIP describes the extension of the Kafka Streams'
> > > metrics
> > > > >> to
> > > > >> > > > include
> > > > >> > > > > >> RocksDB's internal statistics.
> > > > >> > > > > >>
> > > > >> > > > > >> Please have a look at it and let me know what you
> think.
> > > Since
> > > > >> I
> > > > >> > am
> > > > >> > > > not a
> > > > >> > > > > >> RocksDB expert, I am thankful for any additional pair
> of
> > > eyes
> > > > >> that
> > > > >> > > > > >> evaluates this KIP.
> > > > >> > > > > >>
> > > > >> > > > > >>
> > > > >> > > > > >>
> > > > >> > > >
> > > > >> >
> > > > >>
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-471:+Expose+RocksDB+Metrics+in+Kafka+Streams
> > > > >> > > > > >>
> > > > >> > > > > >> Best regards,
> > > > >> > > > > >> Bruno
> > > > >> > > > > >>
> > > > >> > > > > >
> > > > >> > > >
> > > > >> >
> > > > >>
> > > > >>
> > > > >> --
> > > > >> *Dongjin Lee*
> > > > >>
> > > > >> *A hitchhiker in the mathematical world.*
> > > > >> *github:  <http://goog_969573159/>github.com/dongjinleekr
> > > > >> <https://github.com/dongjinleekr>linkedin:
> > > > >> kr.linkedin.com/in/dongjinleekr
> > > > >> <https://kr.linkedin.com/in/dongjinleekr>speakerdeck:
> > > > >> speakerdeck.com/dongjin
> > > > >> <https://speakerdeck.com/dongjin>*
> > > > >>
> > > > >
> > > > >
> > > > > --
> > > > > -- Guozhang
> > > > >
> > >
> >
> >
> > --
> > -- Guozhang
>