Anything to add? Is there a Jira ticket for this yet?

On Fri, Apr 20, 2018 at 1:03 PM, Stefan Richter <s.rich...@data-artisans.com
> wrote:

> If estimates are good enough, we should be able to expose something. Would
> still like to double check the guarantees to see if the estimates of
> RocksDB are helpful or could be misleading.
>
>
> Am 20.04.2018 um 11:59 schrieb Juho Autio <juho.au...@rovio.com>:
>
> Thanks. At least for us it doesn't matter how exact the number is. I would
> expect most users to be only interested in monitoring if the total state
> size keeps growing (rapidly), or remains about the same. I suppose all of
> the options that you suggested would satisfy this need?
>
> On Fri, Apr 20, 2018 at 12:53 PM, Stefan Richter <
> s.rich...@data-artisans.com> wrote:
>
>> Hi,
>>
>> for incremental checkpoints, it is only showing the size of the deltas.
>> It would probably also be possible to report the full size, but the current
>> reporting and UI is only supporting to deliver a single value. In general,
>> some things are rather hard to report. For example, for the heap based
>> backend, is the state size the size of the serialized data or the size of
>> the heap objects?
>> Another remark about key count: the key count is easy to determine for
>> the heap based backend, but there is no (efficient) method in RocksDb that
>> gives the key count (because of the way RocksDB works). In this case,
>> afaik, we have the (inefficient) option to iterate all keys and count or
>> use the (efficient) estimated key count is supported by RocksDB.
>>
>> Best,
>> Stefan
>>
>>
>> Am 04.01.2018 um 19:23 schrieb Steven Wu <stevenz...@gmail.com>:
>>
>> Aljoscha/Stefan,
>>
>> if incremental checkpoint is enabled, I assume the "checkpoint size" is
>> only the delta/incremental size (not the full state size), right?
>>
>> Thanks,
>> Steven
>>
>>
>> On Thu, Jan 4, 2018 at 5:18 AM, Aljoscha Krettek <aljos...@apache.org>
>> wrote:
>>
>>> Hi,
>>>
>>> I'm afraid there is currently no metrics around state. I see that it's
>>> very good to have so I'm putting it on my list of stuff that we should have
>>> at some point.
>>>
>>> One thing that comes to mind is checking the size of checkpoints, which
>>> gives you an indirect way of figuring out how big state is but that's not
>>> very exact, i.e. doesn't give you "number of keys" or some such.
>>>
>>> Best,
>>> Aljoscha
>>>
>>> > On 20. Dec 2017, at 08:09, Netzer, Liron <liron.net...@citi.com>
>>> wrote:
>>> >
>>> > Ufuk, Thanks for replying !
>>> >
>>> > Aljoscha, can you please assist with the questions below?
>>> >
>>> > Thanks,
>>> > Liron
>>> >
>>> > -----Original Message-----
>>> > From: Ufuk Celebi [mailto:u...@apache.org]
>>> > Sent: Friday, December 15, 2017 3:06 PM
>>> > To: Netzer, Liron [ICG-IT]
>>> > Cc: user@flink.apache.org
>>> > Subject: Re: Flink State monitoring
>>> >
>>> > Hey Liron,
>>> >
>>> > unfortunately, there are no built-in metrics related to state. In
>>> general, exposing the actual values as metrics is problematic, but exposing
>>> summary statistics would be a good idea. I'm not aware of a good work
>>> around at the moment that would work in the general case (taking into
>>> account state restore, etc.).
>>> >
>>> > Let me pull in Aljoscha (cc'd) who knows the state backend internals
>>> well.
>>> >
>>> > @Aljoscha:
>>> > 1) Are there any plans to expose keyed state related metrics (like
>>> number of keys)?
>>> > 2) Is there a way to work around the lack of these metrics in 1.3?
>>> >
>>> > – Ufuk
>>> >
>>> > On Thu, Dec 14, 2017 at 10:55 AM, Netzer, Liron <liron.net...@citi.com>
>>> wrote:
>>> >> Hi group,
>>> >>
>>> >>
>>> >>
>>> >> We are using Flink keyed state in several operators.
>>> >>
>>> >> Is there an easy was to expose the data that is stored in the state,
>>> i.e.
>>> >> the key and the values?
>>> >>
>>> >> This is needed for both monitoring as well as debugging. We would like
>>> >> to understand how many key+values are stored in each state and also to
>>> >> view the data itself.
>>> >>
>>> >> I know that there is the "Queryable state" option, but this is still
>>> >> in Beta, and doesn't really give us what we want easily.
>>> >>
>>> >>
>>> >>
>>> >>
>>> >>
>>> >> *We are using Flink 1.3.2 with Java.
>>> >>
>>> >>
>>> >>
>>> >> Thanks,
>>> >>
>>> >> Liron
>>>
>>>
>>
>>
>
>

Reply via email to