From dev-return-8980-archive-asf-public=cust-asf.ponee.io@beam.apache.org Fri Apr 6 19:26:32 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id 7F88A180649 for ; Fri, 6 Apr 2018 19:26:31 +0200 (CEST) Received: (qmail 57950 invoked by uid 500); 6 Apr 2018 17:26:30 -0000 Mailing-List: contact dev-help@beam.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@beam.apache.org Delivered-To: mailing list dev@beam.apache.org Received: (qmail 57929 invoked by uid 99); 6 Apr 2018 17:26:29 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 06 Apr 2018 17:26:29 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id 44958C16DE for ; Fri, 6 Apr 2018 17:26:29 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 1.888 X-Spam-Level: * X-Spam-Status: No, score=1.888 tagged_above=-999 required=6.31 tests=[DKIM_SIGNED=0.1, DKIM_VALID=-0.1, DKIM_VALID_AU=-0.1, HTML_MESSAGE=2, RCVD_IN_DNSWL_NONE=-0.0001, RCVD_IN_MSPIKE_H2=-0.001, SPF_PASS=-0.001, T_RP_MATCHES_RCVD=-0.01] autolearn=disabled Authentication-Results: spamd1-us-west.apache.org (amavisd-new); dkim=pass (2048-bit key) header.d=google.com Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id HIxZGi7t0CM7 for ; Fri, 6 Apr 2018 17:26:27 +0000 (UTC) Received: from mail-io0-f180.google.com (mail-io0-f180.google.com [209.85.223.180]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTPS id 6F46C5FAC2 for ; Fri, 6 Apr 2018 17:26:26 +0000 (UTC) Received: by mail-io0-f180.google.com with SMTP id q84so2554976iod.10 for ; Fri, 06 Apr 2018 10:26:26 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=google.com; s=20161025; h=mime-version:references:in-reply-to:from:date:message-id:subject:to :cc; bh=FE6sq+S8Gm163Hyd4oCCD7AlW3BPAH1ZHUMKczCdb68=; b=C6aCaMGLv4XnH5d9i0cYrunuRqX86XJlgo1Guz0psMnR5H+32yC74PMbEVxls6ZWeW 4ZQ9SHSou+Kalga48dsRBePFvk3c6ct8PMm1WU4t7Pvg1+g4PkwYmCKJfsDBYmvqsw7j ZnHZwEltcDwX5tXPCLGCz3IhGqgx7cV6G+FtaE/lscjI7NOxFC1PBQKA99hkVUVWeUCW FGd+WvUs0cLK0qgeM0J6WINM4Y8Y/g30pytAv9fT2tNMfyLBE13PdoKpdKw9B71TUOEl Pmv4lPWDurjziKi/eo8F+0jLmJApDHYubig0eQs+IVWu9xT4tqeCrQTedUg45q9Bm5T7 hwUg== X-Google-DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=1e100.net; s=20161025; h=x-gm-message-state:mime-version:references:in-reply-to:from:date :message-id:subject:to:cc; bh=FE6sq+S8Gm163Hyd4oCCD7AlW3BPAH1ZHUMKczCdb68=; b=dRjuniTpHd6gMCal+mFgaasAJoHFaH39cJGSqDuzfKOdnjtzglzCatgAo8V/QL1kvV /yOKwM4KNPgM+uadP18Vknp7q2FxYvh7Vev8KSLkl7hr3HKkMAU1eqdQ48viLo8q8dRL QohaNNNSR3Bi6qV69ZYR1ZRhB88Ar2/uKRAXcLxAbVlemQOfYHh/uDwKpzDw3Kd9Vl07 I42VEb1lnBOcTyCm+hta2YOu0cmR87abk/A/OCKbSXpo5GLsXIXizt/1dNs1ePu9IX9z LIiXnjx6hyMX+yH17Q9TG5Ds7vUjN7mkKETgp/yQQBebcH6FNiGeqA3pFY7ezUqK2V3A 0vcg== X-Gm-Message-State: AElRT7FA2Lx5E9TtrVUMZZ4O1YR5AJG4NLkcN289c9zvzIT0YrCOr+up ZlN3ZGYN5kzMz5Mnxf2mFdrI+eYWy4U2dfsOCjsVqXAE X-Google-Smtp-Source: AIpwx494MQTLgcWpONcTMHsvhCcKF5OS/4CRXx5KLW/JfVCmzrmvDRPk2Pd0/4ApMMw9KJCPJ8kUf8i7BO45uyDXe4g= X-Received: by 10.107.184.6 with SMTP id i6mr25211035iof.98.1523035584676; Fri, 06 Apr 2018 10:26:24 -0700 (PDT) MIME-Version: 1.0 References: In-Reply-To: From: Raghu Angadi Date: Fri, 06 Apr 2018 17:26:14 +0000 Message-ID: Subject: Re: About the Gauge metric API To: dev@beam.apache.org Cc: bchambers@apache.org, Andrea Foegler Content-Type: multipart/alternative; boundary="94eb2c0769fa01e5da0569315bec" --94eb2c0769fa01e5da0569315bec Content-Type: text/plain; charset="UTF-8" I am not opposed to removing other data types, though they are extra convenience for user. In Scott's example above, if the metric is a counter, what are the guarantees provided? E.g. would it match the global count using GBK? If yes, then gauges (especially per-key gauges) can be very useful too (e.g. backlog for each Kafka partition/split). On Fri, Apr 6, 2018 at 10:01 AM Robert Bradshaw wrote: > 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 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 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 >>> 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 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 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 >>>>>> 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 >>>>>>> >>>>>>> >>>>>> -- >>>>>> >>>>>> >>>>>> Got feedback? http://go/swegner-feedback >>>>>> >>>>> -- >>> Got feedback? go/pabloem-feedback >>> >>> >> --94eb2c0769fa01e5da0569315bec Content-Type: text/html; charset="UTF-8" Content-Transfer-Encoding: quoted-printable
I am not opposed to removing other data types, though= they are extra convenience for user.

In Scott's ex= ample above, if the metric is a counter, what are the guarantees provided? = E.g. would it match the global count using GBK? If yes, then gauges (especi= ally per-key gauges) can be very useful too (e.g. backlog for each Kafka pa= rtition/split).

On Fri= , Apr 6, 2018 at 10:01 AM Robert Bradshaw <robertwb@google.com> wrote:
A String API makes it clear(er) that the values = will not be aggregated in any way across workers. I don't think retaini= ng 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.=C2=A0

On Fri,= Apr 6, 2018 at 9:46 AM Raghu Angadi <rangadi@google.com> wrote:
I would be in favor of replacing the existi= ng Gauge.set(long) API with the String version and removing the old one. Th= is 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 mak= es it less clear which one users should choose.
Supporting additional data types sounds good. But the above sta= tes string API will replace the existing API. I do not see how string API m= akes the semantics more clear.=C2=A0=C2=A0Semantically both are same to the= user.=C2=A0

On = Fri, Apr 6, 2018 at 9:31 AM Pablo Estrada <pabloem@google.com> wrote:
Hi Ben : D

Su= re, that's reasonable. And perhaps I started the discussion in the wron= g direction. I'm not questioning the utility of Gauge metrics.=C2=A0

What I'm saying is that Beam only supports integ= ers,, but Gauges are aggregated by dropping old values depending on their u= pdate 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 &l= t;bchambers@apach= e.org> wrote:
See for = instance how gauge metrics are handled in Prometheus, Datadog and Stackdriv= er monitoring. Gauges are perfect for use in distributed systems, they just= need to be properly labeled. Perhaps we should apply a default tag or allo= w users to specify one.

On Fri, Apr 6, 2018, 9:14 AM Ben Chambers <bchambers@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 us= age" that need to hold current value.

On Fri, Apr 6, 2018, 9:00 AM Scott Wegner <swegner@google.com> wr= ote:
+1 on the pro= posal to support a "String" gauge.=C2=A0

To ex= pand 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 m= any workers, each worker will have its own local Gauge metric and its updat= es 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 =3D 0;
@ProcessElement
public v= oid processElement(ProcessContext c) {
=C2=A0 myGauge.set(++count= );
=C2=A0 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 nu= mber of elements processed on one of the workers". (The correct implem= entation uses a Counter metric).

I would be in fav= or of replacing the existing Gauge.set(long) API with the String version an= d 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 w= ould retain the potential confusion. It's better to simplify the API su= rface: having two APIs makes it less clear which one users should choose.

On Fri, Apr 6, 20= 18 at 8:28 AM Pablo Estrada <pabloem@google.com> wrote:
Hello all,
As I was working on adding su= pport for Gauges in Dataflow, some noted that Gauge is a fairly unusual kin= d of metric for a distributed environment, since many workers will report d= ifferent values and stomp on each other's all the time.

<= /div>
We also looked at Flink and Dropwizard Gauge metrics [1][2], and = we found that these use generics, and Flink explicitly mentions that a toSt= ring implementation is required[3].

With that in m= ind, I'm thinking that it might make sense to 1) expand Gauge to suppor= t string values (keep int-based API for backwards compatibility), and migra= te it to use string behind the covers.

What does e= veryone think about this?

Best
-P.
=

JIRA issue for Gauge metrics -=C2= =A0https://issues.apache.org/jira/browse/BEAM-1616=C2=A0
= --
Got feedback? go/pabloem-feedba= ck
--
--
Got feedback? go/pabloem-feedback<= /div>
--94eb2c0769fa01e5da0569315bec--