+1 on the proposal to support a "String" gauge.

To expand a bit, the current API doesn't make it clear that the gauge value
is based on local state. If a runner chooses to parallelize a DoFn across
many workers, each worker will have its own local Gauge metric and its
updates will overwrite other values. For example, from the API it looks
like you could use a gauge to implement your own element count metric:

long count = 0;
@ProcessElement
public void processElement(ProcessContext c) {
  myGauge.set(++count);
  c.output(c.element());
}

This looks correct, but each worker has their own local 'count' field, and
gauge metric updates from parallel workers will overwrite each other rather
than get aggregated. So the final value would be "the number of elements
processed on one of the workers". (The correct implementation uses a
Counter metric).

I would be in favor of replacing the existing Gauge.set(long) API with the
String version and removing the old one. This would be a breaking change.
However this is a relatively new API and is still marked @Experimental.
Keeping the old API would retain the potential confusion. It's better to
simplify the API surface: having two APIs makes it less clear which one
users should choose.

On Fri, Apr 6, 2018 at 8:28 AM Pablo Estrada <pabl...@google.com> wrote:

> Hello all,
> As I was working on adding support for Gauges in Dataflow, some noted that
> Gauge is a fairly unusual kind of metric for a distributed environment,
> since many workers will report different values and stomp on each other's
> all the time.
>
> We also looked at Flink and Dropwizard Gauge metrics [1][2], and we found
> that these use generics, and Flink explicitly mentions that a toString
> implementation is required[3].
>
> With that in mind, I'm thinking that it might make sense to 1) expand
> Gauge to support string values (keep int-based API for backwards
> compatibility), and migrate it to use string behind the covers.
>
> What does everyone think about this?
>
> Best
> -P.
>
> 1 -
> https://ci.apache.org/projects/flink/flink-docs-release-1.3/monitoring/metrics.html#metric-types
> 2 - https://metrics.dropwizard.io/3.1.0/manual/core/#gauges
> 3 -
> https://github.com/apache/flink/blob/master/docs/monitoring/metrics.md#gauge
> JIRA issue for Gauge metrics -
> https://issues.apache.org/jira/browse/BEAM-1616
> --
> Got feedback? go/pabloem-feedback
> <https://goto.google.com/pabloem-feedback>
>
-- 


Got feedback? http://go/swegner-feedback

Reply via email to