You can use a cumulative distribution function over the sketch at b0, b1,
b2, b3, ... which will tell you the probability that any given value is <=
X. You multiply that probability against the total count (which is also
recorded as part of the sketch) to get an estimate for the number of values
<= X. If you do this for b0, b1, b2, b3, ... you'll then be able to compute
an estimate for the number of values in each bucket. Note that you're
restoring estimates for how large each bucket is and as such there is some
error.

I was suggesting collecting one or the other, not both. Even if we get this
wrong, we can always swap to the other method by defining a new metric URN
and changing the underlying implementation and what is sent but keeping the
user facing API the same.

On Mon, Aug 17, 2020 at 5:13 PM Alex Amato <[email protected]> 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 <[email protected]> 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 <[email protected]> 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 <[email protected]> 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 <[email protected]>
>>>> wrote:
>>>>
>>>>> On Fri, Aug 14, 2020 at 7:35 PM Alex Amato <[email protected]> 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 <[email protected]>
>>>>> 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 <[email protected]>
>>>>> 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 <[email protected]> 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 <[email protected]>
>>>>> 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 <
>>>>> [email protected]> 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 <[email protected]>
>>>>> 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
>>>>> >> >>
>>>>> >> >>
>>>>>
>>>>

Reply via email to