On Thu, Oct 13, 2016 at 2:27 AM Aljoscha Krettek <aljos...@apache.org>
wrote:

I finally found the time to have a look. :-)



The API looks very good! (It's very similar to an API we recently added to

Flink, which is inspired by the same Codahale/Dropwizard metrics).



About the semantics, the "A", "B" and "C" you mention in the doc: doesn't

this mean that we have to keep the metrics in some fault-tolerant way?

Almost in something like the StateInternals, because they should survive

failures and contain the metrics over the successful runs. (Side note: in

Flink the metrics are just "since the last restart from failure" in case of

failures.)


Aggregators previously required behavior like C -- because they were an
output of the step they would only count the values over successful
invocations of each step. The new API clarifies that there are different
ways of implementing this (when the result of a step is committed we can
commit the counter values from that step, which gives us behavior like B or
C) or we can periodically report the current values (so that we include
values from failed bundles as well) which gives us behavior like A.

Clarifying that there are the different kinds of values is useful
(sometimes you want to compare A vs. C to see how much was "wasted" on
retries) and allows the runner to provide the values it can, and let the
user deal with the fact that the others aren't available.

But yes, to implement all 3 (A, B and C) some interaction with the runner
is necessary.


About querying the metrics, what we have mostly seen is that people want to

integrate metrics into a Metrics system that they already have in place.

They use Graphite, StatsD or simply JMX for this. In Flink we provide an

API for reporters that users can plug in to export the metrics to their

system of choice. I'm sure some people will like the option of having the

metrics queryable on the PipelineResult but I would assume that for most

production use cases integration with a metrics system is more important.


This is an interesting point. Querying was included because (1) we didn't
want to regress the functionality available from Aggregators too much and
(2) it is useful for tests (and logic) to be able to run a pipeline and
check on the counts.


Regarding removal of Aggregators I'm for B, to quote Saint Exupéry:

  "It seems that perfection is attained not when there is nothing more to

add, but when there is nothing more to remove."





> +1 to the new metrics design. I strongly favor B as well.

>

> On Wed, Oct 12, 2016 at 10:54 AM, Kenneth Knowles

> <k...@google.com.invalid> wrote:

> > Correction: In my eagerness to see the end of aggregators, I mistook the

> > intention. Both A and B leave aggregators in place until there is a

> > replacement. In which case, I am strongly in favor of B. As soon as we

> can

> > remove aggregators, I think we should.

> >

> > On Wed, Oct 12, 2016 at 10:48 AM Kenneth Knowles <k...@google.com> wrote:

> >

> >> Huzzah! This is IMO a really great change. I agree that we can get

> >> something in to allow work to continue, and improve the API as we
learn.

> >>

> >> On Wed, Oct 12, 2016 at 10:20 AM Ben Chambers

> <bchamb...@google.com.invalid>

> >> wrote:

> >>

> >> 3. One open question is what to do with Aggregators. In the doc I

> mentioned

> >>

> >> that long term I'd like to consider whether we can improve Aggregators

> to

> >> be a better fit for the model by supporting windowing and allowing them

> to

> >> serve as input for future steps. In the interim it's not clear what we

> >> should do with them. The two obvious (and extreme) options seem to be:

> >>

> >>

> >>

> >>   Option A: Do nothing, leave aggregators as they are until we revisit.

> >>

> >>

> >>   Option B: Remove aggregators from the SDK until we revisit.

> >>

> >> I'd like to suggest removing Aggregators once the existing runners have

> >> reasonable support for Metrics. Doing so reduces the surface area we

> need

> >> to maintain/support and simplifies other changes being made. It will

> also

> >> allow us to revisit them from a clean slate.

> >>

> >>

> >> +1 to removing aggregators, either of A or B. The new metrics design

> >> addresses aggregator use cases as well or better.

> >>

> >> So A vs B is a choice of whether we have a gap with no aggregator or

> >> metrics-like functionality. I think that is perhaps a bit of a bummer

> for

> >> users, and we will likely port over the runner code for it, so we

> wouldn't

> >> want to actually delete it, right? Can we do it in a week or two?

> >>

> >> One thing motivating me to do this quickly: Currently the new DoFn does

> >> not have its own implementation of aggregators, but leverages that of

> >> OldDoFn, so we cannot remove OldDoFn until either (1) new DoFn

> >> re-implements the aggregator instantiation and worker-side delegation

> (not

> >> hard, but it is throwaway code) or (2) aggregators are removed. This

> >> dependency also makes running the new DoFn directly (required for the

> state

> >> API) a bit more annoying.

> >>

>

Reply via email to