A String API makes it clear(er) that the values will not be aggregated in
any way across workers. I don't think retaining both APIs (except for
possibly some short migration period) worthwhile. On another note, I still
find the distributed gague API to be a bit odd in general.

On Fri, Apr 6, 2018 at 9:46 AM Raghu Angadi <rang...@google.com> wrote:

> 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.
>
>
> Supporting additional data types sounds good. But the above states string
> API will replace the existing API. I do not see how string API makes the
> semantics more clear.  Semantically both are same to the user.
>
> On Fri, Apr 6, 2018 at 9:31 AM Pablo Estrada <pabl...@google.com> wrote:
>
>> Hi Ben : D
>>
>> Sure, that's reasonable. And perhaps I started the discussion in the
>> wrong direction. I'm not questioning the utility of Gauge metrics.
>>
>> What I'm saying is that Beam only supports integers,, but Gauges are
>> aggregated by dropping old values depending on their update times; so it
>> might be desirable to not restrict the data type to just integers.
>>
>> -P.
>>
>> On Fri, Apr 6, 2018 at 9:19 AM Ben Chambers <bchamb...@apache.org> wrote:
>>
>>> See for instance how gauge metrics are handled in Prometheus, Datadog
>>> and Stackdriver monitoring. Gauges are perfect for use in distributed
>>> systems, they just need to be properly labeled. Perhaps we should apply a
>>> default tag or allow users to specify one.
>>>
>>> On Fri, Apr 6, 2018, 9:14 AM Ben Chambers <bchamb...@apache.org> wrote:
>>>
>>>> Some metrics backend label the value, for instance with the worker that
>>>> sent it. Then the aggregation is latest per label. This makes it useful for
>>>> holding values such as "memory usage" that need to hold current value.
>>>>
>>>> On Fri, Apr 6, 2018, 9:00 AM Scott Wegner <sweg...@google.com> wrote:
>>>>
>>>>> +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
>>>>>
>>>> --
>> Got feedback? go/pabloem-feedback
>> <https://goto.google.com/pabloem-feedback>
>>
>

Reply via email to