Hi, I'm currently in holidays but I'll put some thought into this and give my comments once I get back.
Aljoscha On Wed, Oct 5, 2016, 21:51 Ben Chambers <bchamb...@google.com.invalid> wrote: > To provide some more background I threw together a quick doc outlining my > current thinking for this Metrics API. You can find it at > http://s.apache.org/beam-metrics-api. > > The first PR (https://github.com/apache/incubator-beam/pull/1024) > introducing these APIs for the direct runner is hopefully nearing > completion. If there are no objections, I'd like to check it in and start > working on hooking this up to other runners to flesh out how this will > interact with them. We can continue to iterate on the API and concepts in > the doc and create follow-up PRs for any changes we'd like to make. > > As always, let me know if there are any questions or comments! > > -- Ben > > On Wed, Sep 28, 2016 at 5:05 PM Ben Chambers <bchamb...@google.com> wrote: > > I started looking at BEAM-147: “Rename Aggregator to [P]Metric”. Rather > than renaming the existing concept I’d like to introduce Metrics as a > simpler mechanism to provide information during pipeline execution (I have > updated the issue accordingly). > > Here is what I'm thinking would lead to a simpler API focused on reporting > metrics about pipeline execution: > > 1. > > Rather than support arbitrary Combine functions, Metrics support a set > of specific aggregations with documented use-cases (eg., Counter, Meter, > Distribution, etc.) and an API inspired by the Dropwizard Metrics > library. > 2. > > Rather than requiring declaration during pipeline construction (like > Aggregators) Metrics allow declaration at any point because it is > easier to > use. > 3. > > Metrics provide more documented flexibility in how runners support them, > by allowing each runner to provide different details about metrics and > support different kinds of metrics, while clearly documenting what the > kinds are and what should happen if they aren’t supported. This allows > users to use metrics in a reliable way even though runners may implement > them differently > > > # What does the Metrics API look like? > > The API for using metrics would be relatively simple: > > // Metrics can be used as fields: > > private final Counter cnt = Metrics.counter(“mycode”, “odd-elements”); > > @ProcessElement > > public void processElement(ProcessContext c) { > > if (c.element() % 2 == 1) { > > cnt.inc(); > > } > > // Metrics can be created dynamically: > > Metrics.distribution(“mycode”, “elements”).report(c.element()); > > ... > > } > > # What Kinds of Metrics could there be? > > There are many kinds of metrics that seem like they could be useful. We > could eventually support metrics like the following: > > - > > Counter: Can be incremented/decremented. Will be part of the initial > implementation. > - > > Distribution: Values can be reported and various statistics are > reported. The initial implementation will support “easy” statistics like > MIN/MAX/MEAN/SUM/COUNT. We’d like to support quantiles in the future to > make this more comparable to Dropwizard’s Histogram. > - > > (Future) Meter: Method to indicate something happened. Computes the rate > of occurrences. > - > > (Future) Timer: A meter measuring how often something happens plus a > distribution of how long it took each time. > - > > (Future) Frequent Elements: Reports values that occurred more than N% of > the time. > > > # What are the next steps? > > I’ve started work prototyping the new API by implementing it for the Java > DirectRunner. To see an example pipeline that reports a Counter and a > Distribution, take a look at the first PR > https://github.com/apache/incubator-beam/pull/1024 > > # Where does that leave Aggregators? > Hopefully, this new Metrics API addresses the goals of monitoring a > pipeline more cleanly than Aggregators. In the long term, it would be good > to make Aggregators a more complete participant in the model, by adding > support for windowing and allowing the results to be used as input to later > steps in the pipeline. Or to make them completely unnecessary by making it > easy to use side-outputs with the new reflective DoFn approach. Once > Metrics are available, we may want to deprecate or remove Aggregators until > we’re ready to figure out what the right API is. >