Hi guys, I've drafted a simple one-page doc for my proposal, please help review, comments are welcome: https://docs.google.com/document/d/1-35iyCIJ9P4EQONlakgXBFRGUYoOLanq2Uf2sw5EjJw/edit?usp=sharing
On Sat, Jun 24, 2017 at 1:31 AM, Ben Chambers <[email protected]> wrote: > I think the distinction between metrics being reported (the API we give > users to create Metrics) from getting metrics out of (currently either > using the PipelineResult to query results or connecting to an external > metric store) is important. > > There is an additional distinction to be made in data processing systems -- > specifically whether the metric value represents a physical quantity such > as the number of RPCs issued during processing which should be counted > across all attempts at processing elements or a logical value such as the > number of elements processed that should only be aggregated across the > successful attempt at an element. > > This distinction requires deeper runner involvement since that is the only > point that knows for sure whether an element has been successfully > processed. It is also the only form of Metrics that are not subject to > potential data loss (over or under counting). > > Allowing Metrics to preserve this distinction and each runner to choose > which of committed and attempted Metrics to support (or possibly even both) > is part of the reason why the existing API is the way it is. > > On Fri, Jun 23, 2017 at 2:41 AM Ismaël Mejía <[email protected]> wrote: > > > That seems like a great idea (improving the current metrics design), I > > suppose there is a tradeoff between complexity and simplicity, and > > when I read the design document I think that some design decisions > > were done for the sake of simplicity, however as dropwizard is the > > 'de-facto' standard for metrics (at least in the java world), then it > > makes sense to align more with it, and that also reminds me that Aviem > > also wanted to add dropwizard's EWMA to the metrics API, so there is > > still some work to do. > > > > > > On Fri, Jun 23, 2017 at 11:00 AM, Cody Innowhere <[email protected]> > > wrote: > > > Hi Ismaël, > > > Yes Distribution is similar to codahale's Histogram without the > > quantiles, > > > and what I meant "adding support of Histogram" might be extending > > > Distribution > > > so that quantiles can be supported. > > > I think in metrics area dropwizard metrics is more or less a standard > and > > > many frameworks have direct support for this including Spark, Flink and > > > JStorm > > > (Well I happen to be the developer of JStorm metrics and our internal > > alert > > > engine), and you're right if beam metrics are compatible with > dropwizard > > we > > > can surely benefit from it. > > > > > > I've also read the design doc and IMHO it's not easy to support > > > Meter/Histogram (currently Distribution is a bit too simple). I'm > > thinking > > > about adding full > > > support of dropwizard metrics and will come up with a doc later so that > > we > > > can discuss this in detail. > > > > > > On Fri, Jun 23, 2017 at 4:30 PM, Ismaël Mejía <[email protected]> > wrote: > > > > > >> Cody not sure if I follow, but isn't Distribution on Beam similar to > > >> codahale/dropwizard's HIstogram (without the quantiles) ? > > >> > > >> Meters are also in the plan but not implemented yet, see the Metrics > > >> design doc: > > >> https://s.apache.org/beam-metrics-api > > >> > > >> If I understand what you want is to have some sort of compatibility > > >> with dropwizard so we can benefit of their sinks ? Is this or I am > > >> misreading it, if so, that would be neat, however the only problem is > > >> the way the aggregation phase passes on Beam because of distribution > > >> vs dropwizard (not sure if they have some implementation that takes > > >> distribution into account). > > >> > > >> Improving metrics is in the agenda and contributions are welcomed > > >> because the API is still evolving and we can have new ideas as part of > > >> it. > > >> > > >> Regards, > > >> Ismaël > > >> > > >> > > >> On Fri, Jun 23, 2017 at 9:29 AM, Cody Innowhere <[email protected]> > > >> wrote: > > >> > Yes I agree with you and sorry for messing them together in this > > >> discussion. > > >> > I just wonder if someone plans to support Meters/Histograms in the > > near > > >> > future. If so, we might need to modify metrics a bit in beam sdk > IMHO, > > >> > that's the reason I started this discussion. > > >> > > > >> > On Fri, Jun 23, 2017 at 3:21 PM, Jean-Baptiste Onofré < > > [email protected]> > > >> > wrote: > > >> > > > >> >> Hi Codi, > > >> >> > > >> >> I think there are two "big" topics around metrics: > > >> >> > > >> >> - what we collect > > >> >> - where we send the collected data > > >> >> > > >> >> The "generic metric sink" (BEAM-2456) is for the later: we don't > > really > > >> >> change/touch the collected data (or maybe just in case of data > > format) > > >> we > > >> >> send to the sink. > > >> >> > > >> >> The Meters/Histograms is both more the collected data IMHO. > > >> >> > > >> >> Regards > > >> >> JB > > >> >> > > >> >> > > >> >> On 06/23/2017 04:09 AM, Cody Innowhere wrote: > > >> >> > > >> >>> Hi JB, > > >> >>> Glad to hear that. > > >> >>> Still, I'm thinking about adding support of Meters & > > Histograms(maybe > > >> >>> extending Distribution). As the discussion mentions, problem is > that > > >> >>> Meter/Histogram > > >> >>> cannot be updated directly in current way because their internal > > data > > >> >>> decays after time. Do you plan to refactor current implementation > so > > >> that > > >> >>> they can be supported while working on the generic metric sink? > > >> >>> > > >> >>> On Thu, Jun 22, 2017 at 9:37 PM, Jean-Baptiste Onofré < > > [email protected] > > >> > > > >> >>> wrote: > > >> >>> > > >> >>> Hi > > >> >>>> > > >> >>>> Agree with Aviem and yes actually I'm working on a generic metric > > >> sink. I > > >> >>>> created a Jira about that. I'm off today, I will send some > details > > >> asap. > > >> >>>> > > >> >>>> Regards > > >> >>>> JB > > >> >>>> > > >> >>>> On Jun 22, 2017, 15:16, at 15:16, Aviem Zur <[email protected]> > > >> wrote: > > >> >>>> > > >> >>>>> Hi Cody, > > >> >>>>> > > >> >>>>> Some of the runners have their own metrics sink, for example > Spark > > >> >>>>> runner > > >> >>>>> uses Spark's metrics sink which you can configure to send the > > metrics > > >> >>>>> to > > >> >>>>> backends such as Graphite. > > >> >>>>> > > >> >>>>> There have been ideas floating around for a Beam metrics sink > > >> extension > > >> >>>>> which will allow users to send Beam metrics to various metrics > > >> >>>>> backends, I > > >> >>>>> believe @JB is working on something along these lines. > > >> >>>>> > > >> >>>>> On Thu, Jun 22, 2017 at 2:00 PM Cody Innowhere < > > [email protected]> > > >> >>>>> wrote: > > >> >>>>> > > >> >>>>> Hi guys, > > >> >>>>>> Currently metrics are implemented in runners/core as > CounterCell, > > >> >>>>>> GaugeCell, DistributionCell, etc. If we want to send metrics to > > >> >>>>>> > > >> >>>>> external > > >> >>>>> > > >> >>>>>> systems via metrics reporter, we would have to define another > > set of > > >> >>>>>> metrics, say, codahale metrics, and update codahale metrics > > >> >>>>>> > > >> >>>>> periodically > > >> >>>>> > > >> >>>>>> with beam sdk metrics, which is inconvenient and inefficient. > > >> >>>>>> > > >> >>>>>> Another problem is that Meter/Histogram cannot be updated > > directly > > >> in > > >> >>>>>> > > >> >>>>> this > > >> >>>>> > > >> >>>>>> way because their internal data decays after time. > > >> >>>>>> > > >> >>>>>> My opinion would be bridge beam sdk metrics to underlying > > runners so > > >> >>>>>> > > >> >>>>> that > > >> >>>>> > > >> >>>>>> updates would directly apply to underlying runners (Flink, > Spark, > > >> >>>>>> > > >> >>>>> etc) > > >> >>>>> > > >> >>>>>> without conversion. > > >> >>>>>> > > >> >>>>>> Specifically, currently we already delegate > > >> >>>>>> Metrics.counter/gauge/distribution to > > >> >>>>>> > > >> >>>>> DelegatingCounter/Gauge/Distribution, > > >> >>>>> > > >> >>>>>> which uses MetricsContainer to store the actual metrics with > the > > >> >>>>>> implementation of MetricsContainerImpl. If we can add an API in > > >> >>>>>> MetricsEnvironment to allow runners to override the default > > >> >>>>>> > > >> >>>>> implementation, > > >> >>>>> > > >> >>>>>> say, for flink, we have FlinkMetricsContainerImpl, then all > > metric > > >> >>>>>> > > >> >>>>> updates > > >> >>>>> > > >> >>>>>> will directly apply to metrics in FlinkMetricsContainerImpl > > without > > >> >>>>>> intermediate conversion and updates. And since the metrics are > > >> >>>>>> runner-specific, it would be a lot easier to support metrics > > >> >>>>>> > > >> >>>>> reporters as > > >> >>>>> > > >> >>>>>> well as Meters/Histograms. > > >> >>>>>> > > >> >>>>>> What do you think? > > >> >>>>>> > > >> >>>>>> > > >> >>>> > > >> >>> > > >> >> -- > > >> >> Jean-Baptiste Onofré > > >> >> [email protected] > > >> >> http://blog.nanthrax.net > > >> >> Talend - http://www.talend.com > > >> >> > > >> > > >
