You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@beam.apache.org by Alex Amato <aj...@google.com> on 2020/09/08 20:06:19 UTC

Re: Percentile metrics in Beam

I've updated the Histogram Style Metrics design
<https://s.apache.org/beam-histogram-metrics> for the FN API based, with a
section exploring the Moment Sketch. PTAL at the “Collect Moment Sketch
Variables Instead of Bucket Counts” section, and see the assessment. LMK
what you think :)


Date

Changes

Sept 8, 2020

   -

   Added alternative section: “Collect Moment Sketch Variables Instead of
   Bucket Counts” (recommend not pursuing, due to opposing trade offs and
   significant implementation/maintenance challenge. But may be worth pursuing
   in a future MonitoringInfo type).
   -

   Add distribution variables: min, max, sum, count
   -

   Added alternative section: “Update all distribution metrics to be
   Histograms” (recommend not pursuing, update to histogramDistribution on a
   case by case, due to performance concerns).

May 15, 2020

   -

   Completed review with beam dev list.


------
Re @Lukasz Cwik <lc...@google.com>
I saw that code you linked, which calls into
linearTimeIncrementHistogramCounters, but got confused a bit more when I
tried to dive into the implementation.
(Relevant, since I would need to port parts of this to python for the SDK,
and C++ for the RunnerHarness side)

I asked a data science peer to help me understand this a bit more. I was
trying to get the equation for the CDF. And added a section on how to
derive the CDF in the doc.
If I understand correctly, we need to calculate the theta values (which
depend on the current moment sketch variables) to calculate the CDF.
Which can then be used to estimate bucket counts, with an integral equation
in the paper.





On Tue, Aug 18, 2020 at 11:35 AM Luke Cwik <lc...@google.com> wrote:

> getPMForCDF[1] seems to return a CDF and you can choose the split points
> (b0, b1, b2, ...).
>
> 1:
> https://github.com/stanford-futuredata/msketch/blob/cf4e49e860761f48ebdeb00f650ce997c46073e2/javamsketch/quantilebench/src/main/java/yahoo/DoublesPmfCdfImpl.java#L16
>
> On Tue, Aug 18, 2020 at 11:20 AM Alex Amato <aj...@google.com> wrote:
>
>> I'm a bit confused, are you sure that it is possible to derive the CDF?
>> Using the moments variables.
>>
>> The linked implementation on github seems to not use a derived CDF
>> equation, but instead using some sampling technique (which I can't fully
>> grasp yet) to estimate how many elements are in each bucket.
>>
>> linearTimeIncrementHistogramCounters
>>
>> https://github.com/stanford-futuredata/msketch/blob/cf4e49e860761f48ebdeb00f650ce997c46073e2/javamsketch/quantilebench/src/main/java/yahoo/DoublesPmfCdfImpl.java#L117
>>
>> Calls into .get() to do some sort of sampling
>>
>> https://github.com/stanford-futuredata/msketch/blob/cf4e49e860761f48ebdeb00f650ce997c46073e2/javamsketch/quantilebench/src/main/java/yahoo/DirectDoublesSketchAccessor.java#L29
>>
>>
>>
>> On Tue, Aug 18, 2020 at 9:52 AM Ke Wu <ke...@gmail.com> wrote:
>>
>>> Hi Alex,
>>>
>>> It is great to know you are working on the metrics. Do you have any
>>> concern if we add a Histogram type metrics in Samza Runner itself for now
>>> so we can start using it before a generic histogram metrics can be
>>> introduced in the Metrics class?
>>>
>>> Best,
>>> Ke
>>>
>>> On Aug 18, 2020, at 12:57 AM, Gleb Kanterov <gl...@spotify.com> wrote:
>>>
>>> Hi Alex,
>>>
>>> I'm not sure about restoring histogram, because the use-case I had in
>>> the past used percentiles. As I understand it, you can approximate
>>> histogram if you know percentiles and total count. E.g. 5% of values fall
>>> into [P95, +INF) bucket, other 5% [P90, P95), etc. I don't understand the
>>> paper well enough to say how it's going to work if given bucket boundaries
>>> happen to include a small number of values. I guess it's a similar kind of
>>> trade-off when we need to choose boundaries if we want to get percentiles
>>> from histogram buckets. I see primarily moment sketch as a method intended
>>> to approximate percentiles, not histogram buckets.
>>>
>>> /Gleb
>>>
>>> On Tue, Aug 18, 2020 at 2:13 AM Alex Amato <aj...@google.com> wrote:
>>>
>>>> Hi Gleb, and Luke
>>>>
>>>> I was reading through the paper, blog and github you linked to. One
>>>> thing I can't figure out is if it's possible to use the Moment Sketch to
>>>> restore an original histogram.
>>>> Given bucket boundaries: b0, b1, b2, b3, ...
>>>> Can we obtain the counts for the number of values inserted each of the
>>>> ranges: [-INF, B0), … [Bi, Bi+1), …
>>>> (This is a requirement I need)
>>>>
>>>> Not be confused with the percentile/threshold based queries discussed
>>>> in the blog.
>>>>
>>>> Luke, were you suggesting collecting both and sending both over the FN
>>>> API wire? I.e. collecting both
>>>>
>>>>    - the variables to represent the Histogram as suggested in
>>>>    https://s.apache.org/beam-histogram-metrics:
>>>>    - In addition to the moment sketch variables
>>>>    <https://blog.acolyer.org/2018/10/31/moment-based-quantile-sketches-for-efficient-high-cardinality-aggregation-queries/>
>>>>    .
>>>>
>>>> I believe that would be feasible, as we would still retain
>>>> the Histogram data. I don't think we can restore the Histograms with just
>>>> the Sketch, if that was the suggestion. Please let me know if I
>>>> misunderstood.
>>>>
>>>> If that's correct, I can write up the benefits and drawbacks I see for
>>>> both approaches.
>>>>
>>>>
>>>> On Mon, Aug 17, 2020 at 9:23 AM Luke Cwik <lc...@google.com> wrote:
>>>>
>>>>> That is an interesting suggestion to change to use a sketch.
>>>>>
>>>>> I believe having one metric URN that represents all this information
>>>>> grouped together would make sense instead of attempting to aggregate
>>>>> several metrics together. The underlying implementation of using
>>>>> sum/count/max/min would stay the same but we would want a single object
>>>>> that abstracts this complexity away for users as well.
>>>>>
>>>>> On Mon, Aug 17, 2020 at 3:42 AM Gleb Kanterov <gl...@spotify.com>
>>>>> wrote:
>>>>>
>>>>>> Didn't see proposal by Alex before today. I want to add a few more
>>>>>> cents from my side.
>>>>>>
>>>>>> There is a paper Moment-based quantile sketches for efficient high
>>>>>> cardinality aggregation queries [1], a TL;DR that for some N (around 10-20
>>>>>> depending on accuracy) we need to collect SUM(log^N(X)) ... log(X),
>>>>>> COUNT(X), SUM(X), SUM(X^2)... SUM(X^N), MAX(X), MIN(X). Given aggregated
>>>>>> numbers, it uses solver for Chebyshev polynomials to get quantile number,
>>>>>> and there is already Java implementation for it on GitHub [2].
>>>>>>
>>>>>> This way we can express quantiles using existing metric types in
>>>>>> Beam, that can be already done without SDK or runner changes. It can fit
>>>>>> nicely into existing runners and can be abstracted over if needed. I think
>>>>>> this is also one of the best implementations, it has < 1% error rate for
>>>>>> 200 bytes of storage, and quite efficient to compute. Did we consider using
>>>>>> that?
>>>>>>
>>>>>> [1]:
>>>>>> https://blog.acolyer.org/2018/10/31/moment-based-quantile-sketches-for-efficient-high-cardinality-aggregation-queries/
>>>>>> [2]: https://github.com/stanford-futuredata/msketch
>>>>>>
>>>>>> On Sat, Aug 15, 2020 at 6:15 AM Alex Amato <aj...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> The distinction here is that even though these metrics come from
>>>>>>> user space, we still gave them specific URNs, which imply they have a
>>>>>>> specific format, with specific labels, etc.
>>>>>>>
>>>>>>> That is, we won't be packaging them into a USER_HISTOGRAM urn. That
>>>>>>> URN would have less expectation for its format. Today the USER_COUNTER just
>>>>>>> expects like labels (TRANSFORM, NAME, NAMESPACE).
>>>>>>>
>>>>>>> We didn't decide on making a private API. But rather an API
>>>>>>> available to user code for populating metrics with specific labels, and
>>>>>>> specific URNs. The same API could pretty much be used for user
>>>>>>> USER_HISTOGRAM. with a default URN chosen.
>>>>>>> Thats how I see it in my head at the moment.
>>>>>>>
>>>>>>>
>>>>>>> On Fri, Aug 14, 2020 at 8:52 PM Robert Bradshaw <ro...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> On Fri, Aug 14, 2020 at 7:35 PM Alex Amato <aj...@google.com>
>>>>>>>> wrote:
>>>>>>>> >
>>>>>>>> > I am only tackling the specific metrics covered in (for the
>>>>>>>> python SDK first, then Java). To collect latency of IO API RPCS, and store
>>>>>>>> it in a histogram.
>>>>>>>> > https://s.apache.org/beam-gcp-debuggability
>>>>>>>> >
>>>>>>>> > User histogram metrics are unfunded, as far as I know. But you
>>>>>>>> should be able to extend what I do for that project to the user metric use
>>>>>>>> case. I agree, it won't be much more work to support that. I designed the
>>>>>>>> histogram with the user histogram case in mind.
>>>>>>>>
>>>>>>>> From the portability point of view, all metrics generated in users
>>>>>>>> code (and SDK-side IOs are "user code") are user metrics. But
>>>>>>>> regardless of how things are named, once we have histogram metrics
>>>>>>>> crossing the FnAPI boundary all the infrastructure will be in place.
>>>>>>>> (At least the plan as I understand it shouldn't use private APIs
>>>>>>>> accessible only by the various IOs but not other SDK-level code.)
>>>>>>>>
>>>>>>>> > On Fri, Aug 14, 2020 at 5:47 PM Robert Bradshaw <
>>>>>>>> robertwb@google.com> wrote:
>>>>>>>> >>
>>>>>>>> >> Once histograms are implemented in the SDK(s) (Alex, you're
>>>>>>>> tackling
>>>>>>>> >> this, right?) it shoudn't be much work to update the Samza
>>>>>>>> worker code
>>>>>>>> >> to publish these via the Samza runner APIs (in parallel with
>>>>>>>> Alex's
>>>>>>>> >> work to do the same on Dataflow).
>>>>>>>> >>
>>>>>>>> >> On Fri, Aug 14, 2020 at 5:35 PM Alex Amato <aj...@google.com>
>>>>>>>> wrote:
>>>>>>>> >> >
>>>>>>>> >> > Noone has any plans currently to work on adding a generic
>>>>>>>> histogram metric, at the moment.
>>>>>>>> >> >
>>>>>>>> >> > But I will be actively working on adding it for a specific set
>>>>>>>> of metrics in the next quarter or so
>>>>>>>> >> > https://s.apache.org/beam-gcp-debuggability
>>>>>>>> >> >
>>>>>>>> >> > After that work, one could take a look at my PRs for reference
>>>>>>>> to create new metrics using the same histogram. One may wish to implement
>>>>>>>> the UserHistogram use case and use that in the Samza Runner
>>>>>>>> >> >
>>>>>>>> >> >
>>>>>>>> >> >
>>>>>>>> >> >
>>>>>>>> >> > On Fri, Aug 14, 2020 at 5:25 PM Ke Wu <ke...@gmail.com>
>>>>>>>> wrote:
>>>>>>>> >> >>
>>>>>>>> >> >> Thank you Robert and Alex. I am not running a Beam job in
>>>>>>>> Google Cloud but with Samza Runner, so I am wondering if there is any ETA
>>>>>>>> to add the Histogram metrics in Metrics class so it can be mapped to the
>>>>>>>> SamzaHistogram metric to the actual emitting.
>>>>>>>> >> >>
>>>>>>>> >> >> Best,
>>>>>>>> >> >> Ke
>>>>>>>> >> >>
>>>>>>>> >> >> On Aug 14, 2020, at 4:44 PM, Alex Amato <aj...@google.com>
>>>>>>>> wrote:
>>>>>>>> >> >>
>>>>>>>> >> >> One of the plans to use the histogram data is to send it to
>>>>>>>> Google Monitoring to compute estimates of percentiles. This is done using
>>>>>>>> the bucket counts and bucket boundaries.
>>>>>>>> >> >>
>>>>>>>> >> >> Here is a describing of roughly how its calculated.
>>>>>>>> >> >>
>>>>>>>> https://stackoverflow.com/questions/59635115/gcp-console-how-are-percentile-charts-calculated
>>>>>>>> >> >> This is a non exact estimate. But plotting the estimated
>>>>>>>> percentiles over time is often easier to understand and sufficient.
>>>>>>>> >> >> (An alternative is a heatmap chart representing histograms
>>>>>>>> over time. I.e. a histogram for each window of time).
>>>>>>>> >> >>
>>>>>>>> >> >>
>>>>>>>> >> >> On Fri, Aug 14, 2020 at 4:16 PM Robert Bradshaw <
>>>>>>>> robertwb@google.com> wrote:
>>>>>>>> >> >>>
>>>>>>>> >> >>> You may be interested in the propose histogram metrics:
>>>>>>>> >> >>>
>>>>>>>> https://docs.google.com/document/d/1kiNG2BAR-51pRdBCK4-XFmc0WuIkSuBzeb__Zv8owbU/edit
>>>>>>>> >> >>>
>>>>>>>> >> >>> I think it'd be reasonable to add percentiles as its own
>>>>>>>> metric type
>>>>>>>> >> >>> as well. The tricky bit (though there are lots of resources
>>>>>>>> on this)
>>>>>>>> >> >>> is that one would have to publish more than just the
>>>>>>>> percentiles from
>>>>>>>> >> >>> each worker to be able to compute the final percentiles
>>>>>>>> across all
>>>>>>>> >> >>> workers.
>>>>>>>> >> >>>
>>>>>>>> >> >>> On Fri, Aug 14, 2020 at 4:05 PM Ke Wu <ke...@gmail.com>
>>>>>>>> wrote:
>>>>>>>> >> >>> >
>>>>>>>> >> >>> > Hi everyone,
>>>>>>>> >> >>> >
>>>>>>>> >> >>> > I am looking to add percentile metrics (p50, p90 etc) to
>>>>>>>> my beam job but I only find Counter, Gauge and Distribution metrics. I
>>>>>>>> understand that I can calculate percentile metrics in my job itself and use
>>>>>>>> Gauge to emit, however this is not an easy approach. On the other hand,
>>>>>>>> Distribution metrics sounds like the one to go to according to its
>>>>>>>> documentation: "A metric that reports information about the distribution of
>>>>>>>> reported values.”, however it seems that it is intended for SUM, COUNT,
>>>>>>>> MIN, MAX.
>>>>>>>> >> >>> >
>>>>>>>> >> >>> > The question(s) are:
>>>>>>>> >> >>> >
>>>>>>>> >> >>> > 1. is Distribution metric only intended for sum, count,
>>>>>>>> min, max?
>>>>>>>> >> >>> > 2. If Yes, can the documentation be updated to be more
>>>>>>>> specific?
>>>>>>>> >> >>> > 3. Can we add percentiles metric support, such as
>>>>>>>> Histogram, with configurable list of percentiles to emit?
>>>>>>>> >> >>> >
>>>>>>>> >> >>> > Best,
>>>>>>>> >> >>> > Ke
>>>>>>>> >> >>
>>>>>>>> >> >>
>>>>>>>>
>>>>>>>
>>>