You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by zentol <gi...@git.apache.org> on 2016/06/24 10:02:35 UTC

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

GitHub user zentol opened a pull request:

    https://github.com/apache/flink/pull/2158

    [FLINK-4116] Metrics documentation

    

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/zentol/flink metrics_docs

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/2158.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2158
    
----
commit c9a106cf99e225baa7cf13604b7d41bb1b11b80d
Author: zentol <ch...@apache.org>
Date:   2016-06-20T12:56:05Z

    [FLINK-4116] Metrics documentation

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68564208
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter() call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +- `metrics.scope.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +- `metrics.scope.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +There is no restriction on the order of variables.
    +
    +Which format is applied to a metric depends on which entities it was scoped to.
    +The `metrics.scope.operator` format will be applied to all metrics that were scoped to an Operator.
    +The `metrics.scope.tm.job` format will be applied to all metrics that were only scoped to a Job and a TaskManager.
    +
    +This allows you to define explicitly how metrics on every level should be reported.
    +
    +### Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in the `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter)
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010)
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS)
    +
    +By default Flink uses JMX to expose metrics.
    +
    +The port for JMX can be configured by setting the `metrics.jmx.port` key. This parameter expects either a single port
    +or a port range, with the default being 9010-9025. Which port is used in the end is shown in the relevant Job-/TaskManager log.
    +
    +All non-JMXReporters are not part of the distribution and have to be added to the classpath manually. (usually by putting the jar into /lib)
    +
    +You can write your own Reporter by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the Scheduled interface as well.
    +
    +Flink supports the following systems with their respective configuration parameters:
    +
    +#### Ganglia (org.apcahe.flink.metrics.ganglia.GangliaReporter)
    +- `host`
    +- `port`
    +- `dmax`
    +- `tmax`
    +- `ttl`
    +- `addressingMode` (UNICAST/MULTICAST)
    --- End diff --
    
    Which flink module do I have to add for this reporter? Maybe give code snippet which you can easily add to your `pom.xml` file.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68562012
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    --- End diff --
    
    `Counter` in backticks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69893791
  
    --- Diff: docs/apis/metrics.md ---
    @@ -0,0 +1,441 @@
    +---
    +title: "Metrics"
    +# Top-level navigation
    +top-nav-group: apis
    +top-nav-pos: 13
    +top-nav-title: "Metrics"
    +---
    +<!--
    +Licensed to the Apache Software Foundation (ASF) under one
    +or more contributor license agreements.  See the NOTICE file
    +distributed with this work for additional information
    +regarding copyright ownership.  The ASF licenses this file
    +to you under the Apache License, Version 2.0 (the
    +"License"); you may not use this file except in compliance
    +with the License.  You may obtain a copy of the License at
    +
    +  http://www.apache.org/licenses/LICENSE-2.0
    +
    +Unless required by applicable law or agreed to in writing,
    +software distributed under the License is distributed on an
    +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    +KIND, either express or implied.  See the License for the
    +specific language governing permissions and limitations
    +under the License.
    +-->
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +* This will be replaced by the TOC
    +{:toc}
    +
    +## Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.counter = getRuntimeContext()
    +      .getMetricGroup()
    +      .counter("myCounter");
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    this.counter.inc();
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.counter = getRuntimeContext()
    +      .getMetricGroup()
    +      .counter("myCustomCounter", new CustomCounter());
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is no restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    getRuntimeContext()
    +      .getMetricGroup()
    +      .gauge("MyGauge", new Gauge<Integer>() {
    +        @Override
    +        public Integer getValue() {
    +          return valueToExpose;
    +        }
    +      });
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Note that reporters will turn the exposed object into a `String`, which means that a meaningful `toString()` implementation is required.
    +
    +#### Histogram
    +
    +A `Histogram` measures the distribution of long values.
    +You can register one by calling `histogram(String name, Histogram histogram)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.histogram = getRuntimeContext()
    +      .getMetricGroup()
    +      .histogram("myHistogram", new MyHistogram());
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink does not provide a default implementation for `Histogram`, but offers a {% gh_link flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/DropwizardHistogramWrapper.java "Wrapper" %} that allows usage of Codahale/DropWizard histograms.
    +To use this wrapper add the following dependency in your `pom.xml`:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-dropwizard</artifactId>
    +      <version>{{site.version}}</version>
    +</dependency>
    +{% endhighlight %}
    +
    +You can then register a Codahale/DropWizard histogram like this:
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    com.codahale.metrics.Histogram histogram =
    +      new com.codahale.metrics.Histogram(new SlidingWindowReservoir(500));
    +
    +    this.histogram = getRuntimeContext()
    +      .getMetricGroup()
    +      .histogram("myHistogram", new DropWizardHistogramWrapper(histogram));
    +  }
    +}
    +{% endhighlight %}
    +
    +## Scope
    +
    +Every metric is assigned an identifier under which it will be reported that is based on 3 components: the user-provided name when registering the metric, an optional user-defined scope and a system-provided scope.
    +For example, if `A.B` is the sytem scope, `C.D` the user scope and `E` the name, then the identifier for the metric will be `A.B.C.D.E`.
    +
    +### User Scope
    +
    +You can define a user scope by calling either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext()
    +  .getMetricGroup()
    +  .addGroup("MyMetrics")
    +  .counter("myCounter");
    +
    +{% endhighlight %}
    +
    +### System Scope
    +
    +The system scope contains context information about the metric, for example in which task it was registered or what job that task belongs to.
    +
    +Which context information should be included can be configured by setting the following keys in `conf/flink-conf.yaml`.
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "&lt;task_id&gt;") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: &lt;host&gt;.jobmanager
    +  - Applied to all metrics that were scoped to a job manager.
    +- `metrics.scope.jm.job`
    +  - Default: &lt;host&gt;.jobmanager.&lt;job_name&gt;
    +  - Applied to all metrics that were scoped to a job manager and job.
    +- `metrics.scope.tm`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;
    +  - Applied to all metrics that were scoped to a task manager.
    +- `metrics.scope.tm.job`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;
    +  - Applied to all metrics that were scoped to a task manager and job.
    +- `metrics.scope.tm.task`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;.&lt;task_name&gt;.&lt;subtask_index&gt;
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;.&lt;operator_name&gt;.&lt;subtask_index&gt;
    +  - Applied to all metrics that were scoped to an operator.
    +
    +There are no restrictions on the number or order of variables. Variables are case sensitive.
    +
    +The default scope for operator metrics will result in an identifier akin to `localhost.taskmanager.1234.MyJob.MyOperator.0.MyMetric`
    +
    +If you also want to include the task name but omit the task manager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: <host>.<job_name>.<task_name>.<operator_name>.<subtask_index>`
    +
    +This could create the identifier `localhost.MyJob.MySource_->_MyOperator.MyOperator.0.MyMetric`.
    +
    +Note that for this format string an identifier clash can occur should the same job be run multiple times concurrently, which can lead to inconsistent metric data.
    +As such it is advised to either use format strings that provide a certain degree of uniqueness by including IDs (e.g &lt;job_id&gt;)
    +or by assigning unique names to jobs and operators.
    +
    +### List of all Variables
    +
    +- JobManager: &lt;host&gt;
    +- TaskManager: &lt;host&gt;, &lt;tm_id&gt;
    +- Job: &lt;job_id&gt;, &lt;job_name&gt;
    +- Task: &lt;task_id&gt;, &lt;task_name&gt;, &lt;task_attempt_id&gt;, &lt;task_attempt_num&gt;, &lt;subtask_index&gt;
    +- Operator: &lt;operator_name&gt;, &lt;subtask_index&gt;
    +
    +## Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS
    +
    +You can write your own `Reporter` by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the `Scheduled` interface as well.
    +
    +By default Flink uses JMX to expose metrics.
    +All non-JMXReporters are not part of the distribution and have to be added [manually]({{site.baseurl}}/apis/cluster_execution.html#linking-with-modules-not-contained-in-the-binary-distribution).
    +
    +The following sections list the supported reporters.
    +
    +### JMX
    +
    +The port for JMX can be configured by setting the `metrics.jmx.port` key. This parameter expects either a single port
    +or a port range, with the default being 9010-9025. The used port is shown in the relevant job or task manager log.
    +
    +### Ganglia (org.apache.flink.metrics.ganglia.GangliaReporter)
    +Dependency:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-ganglia</artifactId>
    +      <version>{{site.version}}</version>
    +</dependency>
    --- End diff --
    
    I think it is not enough to simply include this dependency to your job in order to use the Ganglia reporter. Instead it should be put in the lib directory, because the jar is required when the cluster is started. It only works if the cluster is started for a single job and the user jars are contained in the system classloader.
    
    I think we should document this properly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by uce <gi...@git.apache.org>.
Github user uce commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69166266
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    +      @Override
    +      public Integer getValue() {
    +        return valueToExpose;
    +      }});
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Histogram
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.histogram = getRuntimeContext().getMetricGroup().histogram("myHistogram", new MyHistogram());
    +    ...
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +    ...
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +This wrapper is contained in the `flink-metrics-dropwizard` module.
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represents the entities it is tied to. By default a metric that is registered in a user function will be scoped to the operator in which the function runs, the task/job it belongs to and the taskManager/host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter()` call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +  - Applied to all metrics that were scoped to a jobmanager.
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +  - Applied to all metrics that were scoped to a jobmanager and job.
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +  - Applied to all metrics that were scoped to a taskmanager.
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +  - Applied to all metrics that were scoped to a taskmanager and job.
    +- `metrics.scope.tm.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +  - Applied to all metrics that were scoped to an operator.
    +
    +Note that for metrics for which multiple formats may apply (like jm and jm.job) the most specific format takes precedence,
    +in this case jm.job.
    +
    +The hierarchical orders are as follows:
    +
    +jm < jm.job 
    +
    +tm < tm.job < tm.task < tm.operator
    +
    +This hierarchy also defines which variables may be accessed. The `tm.operator` format may contain variables for jobs, whereas `tm.job` may not contain
    +variables for operators. There is no restriction on order of variables.
    +
    +There is no restriction on the order of variables.
    +
    +The default scope for operator metrics will result in a metric name akin to `localhost.taskmanager.1234.MyJob.MyOperator.0`
    +
    +If you also want to include the task name, but omit the taskmanager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: \<host>.\<job_name>.\<task_name>.\<operator_name>.\<subtask_index>`
    +
    +This could create the name `localhost.MyJob.MySource_->_MyOperator.MyOperator.0`.
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +### Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in the `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS
    +
    +You can write your own `Reporter` by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the `Scheduled` interface as well.
    +
    +By default Flink uses JMX to expose metrics.
    +All non-JMXReporters are not part of the distribution and have to be added to the classpath manually, either by putting the jar into /lib
    +or including it in the job jar.
    +
    +Flink supports the following systems:
    --- End diff --
    
    I think the `:` and the following sub sections don't go well together. I think either removing the sentence or saying something like `The following sections list the supported reporters` is better


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69093837
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    +      @Override
    +      public Integer getValue() {
    +        return valueToExpose;
    +      }});
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Histogram
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.histogram = getRuntimeContext().getMetricGroup().histogram("myHistogram", new MyHistogram());
    +    ...
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +    ...
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +This wrapper is contained in the `flink-metrics-dropwizard` module.
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represents the entities it is tied to. By default a metric that is registered in a user function will be scoped to the operator in which the function runs, the task/job it belongs to and the taskManager/host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter()` call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +  - Applied to all metrics that were scoped to a jobmanager.
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +  - Applied to all metrics that were scoped to a jobmanager and job.
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +  - Applied to all metrics that were scoped to a taskmanager.
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +  - Applied to all metrics that were scoped to a taskmanager and job.
    +- `metrics.scope.tm.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +  - Applied to all metrics that were scoped to an operator.
    +
    +Note that for metrics for which multiple formats may apply (like jm and jm.job) the most specific format takes precedence,
    +in this case jm.job.
    +
    +The hierarchical orders are as follows:
    +
    +jm < jm.job 
    +
    +tm < tm.job < tm.task < tm.operator
    +
    +This hierarchy also defines which variables may be accessed. The `tm.operator` format may contain variables for jobs, whereas `tm.job` may not contain
    +variables for operators. There is no restriction on order of variables.
    +
    +There is no restriction on the order of variables.
    +
    +The default scope for operator metrics will result in a metric name akin to `localhost.taskmanager.1234.MyJob.MyOperator.0`
    +
    +If you also want to include the task name, but omit the taskmanager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: \<host>.\<job_name>.\<task_name>.\<operator_name>.\<subtask_index>`
    --- End diff --
    
    I used Chrome to look at the docs. Maybe it depends on the browser. Replacing them by `&lt;` and `&gt;` sounds like the right way to go.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69947337
  
    --- Diff: docs/apis/metrics.md ---
    @@ -0,0 +1,441 @@
    +---
    +title: "Metrics"
    +# Top-level navigation
    +top-nav-group: apis
    +top-nav-pos: 13
    +top-nav-title: "Metrics"
    +---
    +<!--
    +Licensed to the Apache Software Foundation (ASF) under one
    +or more contributor license agreements.  See the NOTICE file
    +distributed with this work for additional information
    +regarding copyright ownership.  The ASF licenses this file
    +to you under the Apache License, Version 2.0 (the
    +"License"); you may not use this file except in compliance
    +with the License.  You may obtain a copy of the License at
    +
    +  http://www.apache.org/licenses/LICENSE-2.0
    +
    +Unless required by applicable law or agreed to in writing,
    +software distributed under the License is distributed on an
    +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    +KIND, either express or implied.  See the License for the
    +specific language governing permissions and limitations
    +under the License.
    +-->
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +* This will be replaced by the TOC
    +{:toc}
    +
    +## Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.counter = getRuntimeContext()
    +      .getMetricGroup()
    +      .counter("myCounter");
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    this.counter.inc();
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.counter = getRuntimeContext()
    +      .getMetricGroup()
    +      .counter("myCustomCounter", new CustomCounter());
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is no restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    getRuntimeContext()
    +      .getMetricGroup()
    +      .gauge("MyGauge", new Gauge<Integer>() {
    +        @Override
    +        public Integer getValue() {
    +          return valueToExpose;
    +        }
    +      });
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Note that reporters will turn the exposed object into a `String`, which means that a meaningful `toString()` implementation is required.
    +
    +#### Histogram
    +
    +A `Histogram` measures the distribution of long values.
    +You can register one by calling `histogram(String name, Histogram histogram)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.histogram = getRuntimeContext()
    +      .getMetricGroup()
    +      .histogram("myHistogram", new MyHistogram());
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink does not provide a default implementation for `Histogram`, but offers a {% gh_link flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/DropwizardHistogramWrapper.java "Wrapper" %} that allows usage of Codahale/DropWizard histograms.
    +To use this wrapper add the following dependency in your `pom.xml`:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-dropwizard</artifactId>
    +      <version>{{site.version}}</version>
    +</dependency>
    +{% endhighlight %}
    +
    +You can then register a Codahale/DropWizard histogram like this:
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    com.codahale.metrics.Histogram histogram =
    +      new com.codahale.metrics.Histogram(new SlidingWindowReservoir(500));
    +
    +    this.histogram = getRuntimeContext()
    +      .getMetricGroup()
    +      .histogram("myHistogram", new DropWizardHistogramWrapper(histogram));
    +  }
    +}
    +{% endhighlight %}
    +
    +## Scope
    +
    +Every metric is assigned an identifier under which it will be reported that is based on 3 components: the user-provided name when registering the metric, an optional user-defined scope and a system-provided scope.
    +For example, if `A.B` is the sytem scope, `C.D` the user scope and `E` the name, then the identifier for the metric will be `A.B.C.D.E`.
    +
    +### User Scope
    +
    +You can define a user scope by calling either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext()
    +  .getMetricGroup()
    +  .addGroup("MyMetrics")
    +  .counter("myCounter");
    +
    +{% endhighlight %}
    +
    +### System Scope
    +
    +The system scope contains context information about the metric, for example in which task it was registered or what job that task belongs to.
    +
    +Which context information should be included can be configured by setting the following keys in `conf/flink-conf.yaml`.
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "&lt;task_id&gt;") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: &lt;host&gt;.jobmanager
    +  - Applied to all metrics that were scoped to a job manager.
    +- `metrics.scope.jm.job`
    +  - Default: &lt;host&gt;.jobmanager.&lt;job_name&gt;
    +  - Applied to all metrics that were scoped to a job manager and job.
    +- `metrics.scope.tm`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;
    +  - Applied to all metrics that were scoped to a task manager.
    +- `metrics.scope.tm.job`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;
    +  - Applied to all metrics that were scoped to a task manager and job.
    +- `metrics.scope.tm.task`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;.&lt;task_name&gt;.&lt;subtask_index&gt;
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;.&lt;operator_name&gt;.&lt;subtask_index&gt;
    +  - Applied to all metrics that were scoped to an operator.
    +
    +There are no restrictions on the number or order of variables. Variables are case sensitive.
    +
    +The default scope for operator metrics will result in an identifier akin to `localhost.taskmanager.1234.MyJob.MyOperator.0.MyMetric`
    +
    +If you also want to include the task name but omit the task manager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: <host>.<job_name>.<task_name>.<operator_name>.<subtask_index>`
    +
    +This could create the identifier `localhost.MyJob.MySource_->_MyOperator.MyOperator.0.MyMetric`.
    +
    +Note that for this format string an identifier clash can occur should the same job be run multiple times concurrently, which can lead to inconsistent metric data.
    +As such it is advised to either use format strings that provide a certain degree of uniqueness by including IDs (e.g &lt;job_id&gt;)
    +or by assigning unique names to jobs and operators.
    +
    +### List of all Variables
    +
    +- JobManager: &lt;host&gt;
    +- TaskManager: &lt;host&gt;, &lt;tm_id&gt;
    +- Job: &lt;job_id&gt;, &lt;job_name&gt;
    +- Task: &lt;task_id&gt;, &lt;task_name&gt;, &lt;task_attempt_id&gt;, &lt;task_attempt_num&gt;, &lt;subtask_index&gt;
    +- Operator: &lt;operator_name&gt;, &lt;subtask_index&gt;
    +
    +## Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS
    +
    +You can write your own `Reporter` by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the `Scheduled` interface as well.
    +
    +By default Flink uses JMX to expose metrics.
    +All non-JMXReporters are not part of the distribution and have to be added [manually]({{site.baseurl}}/apis/cluster_execution.html#linking-with-modules-not-contained-in-the-binary-distribution).
    +
    +The following sections list the supported reporters.
    +
    +### JMX
    +
    +The port for JMX can be configured by setting the `metrics.jmx.port` key. This parameter expects either a single port
    +or a port range, with the default being 9010-9025. The used port is shown in the relevant job or task manager log.
    +
    +### Ganglia (org.apache.flink.metrics.ganglia.GangliaReporter)
    +Dependency:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-ganglia</artifactId>
    +      <version>{{site.version}}</version>
    +</dependency>
    --- End diff --
    
    At the moment I fear there is no way around this, since we start the reporters when the cluster is started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/flink/pull/2158


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by uce <gi...@git.apache.org>.
Github user uce commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69166443
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    +      @Override
    +      public Integer getValue() {
    +        return valueToExpose;
    +      }});
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Histogram
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.histogram = getRuntimeContext().getMetricGroup().histogram("myHistogram", new MyHistogram());
    +    ...
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +    ...
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +This wrapper is contained in the `flink-metrics-dropwizard` module.
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represents the entities it is tied to. By default a metric that is registered in a user function will be scoped to the operator in which the function runs, the task/job it belongs to and the taskManager/host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter()` call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +  - Applied to all metrics that were scoped to a jobmanager.
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +  - Applied to all metrics that were scoped to a jobmanager and job.
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +  - Applied to all metrics that were scoped to a taskmanager.
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +  - Applied to all metrics that were scoped to a taskmanager and job.
    +- `metrics.scope.tm.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +  - Applied to all metrics that were scoped to an operator.
    +
    +Note that for metrics for which multiple formats may apply (like jm and jm.job) the most specific format takes precedence,
    +in this case jm.job.
    +
    +The hierarchical orders are as follows:
    +
    +jm < jm.job 
    +
    +tm < tm.job < tm.task < tm.operator
    +
    +This hierarchy also defines which variables may be accessed. The `tm.operator` format may contain variables for jobs, whereas `tm.job` may not contain
    +variables for operators. There is no restriction on order of variables.
    +
    +There is no restriction on the order of variables.
    +
    +The default scope for operator metrics will result in a metric name akin to `localhost.taskmanager.1234.MyJob.MyOperator.0`
    +
    +If you also want to include the task name, but omit the taskmanager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: \<host>.\<job_name>.\<task_name>.\<operator_name>.\<subtask_index>`
    +
    +This could create the name `localhost.MyJob.MySource_->_MyOperator.MyOperator.0`.
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +### Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in the `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS
    +
    +You can write your own `Reporter` by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the `Scheduled` interface as well.
    +
    +By default Flink uses JMX to expose metrics.
    +All non-JMXReporters are not part of the distribution and have to be added to the classpath manually, either by putting the jar into /lib
    +or including it in the job jar.
    +
    +Flink supports the following systems:
    +
    +#### JMX
    +
    +The port for JMX can be configured by setting the `metrics.jmx.port` key. This parameter expects either a single port
    +or a port range, with the default being 9010-9025. The used port is shown in the relevant Job-/TaskManager log.
    +
    +#### Ganglia (org.apache.flink.metrics.ganglia.GangliaReporter)
    +Dependency:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-ganglia</artifactId>
    +      <version>1.1-SNAPSHOT</version>
    --- End diff --
    
    We need to use version variables instead of hard coded values here (and in other places)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69913367
  
    --- Diff: docs/apis/metrics.md ---
    @@ -0,0 +1,441 @@
    +---
    +title: "Metrics"
    +# Top-level navigation
    +top-nav-group: apis
    +top-nav-pos: 13
    +top-nav-title: "Metrics"
    +---
    +<!--
    +Licensed to the Apache Software Foundation (ASF) under one
    +or more contributor license agreements.  See the NOTICE file
    +distributed with this work for additional information
    +regarding copyright ownership.  The ASF licenses this file
    +to you under the Apache License, Version 2.0 (the
    +"License"); you may not use this file except in compliance
    +with the License.  You may obtain a copy of the License at
    +
    +  http://www.apache.org/licenses/LICENSE-2.0
    +
    +Unless required by applicable law or agreed to in writing,
    +software distributed under the License is distributed on an
    +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    +KIND, either express or implied.  See the License for the
    +specific language governing permissions and limitations
    +under the License.
    +-->
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +* This will be replaced by the TOC
    +{:toc}
    +
    +## Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.counter = getRuntimeContext()
    +      .getMetricGroup()
    +      .counter("myCounter");
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    this.counter.inc();
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.counter = getRuntimeContext()
    +      .getMetricGroup()
    +      .counter("myCustomCounter", new CustomCounter());
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is no restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    getRuntimeContext()
    +      .getMetricGroup()
    +      .gauge("MyGauge", new Gauge<Integer>() {
    +        @Override
    +        public Integer getValue() {
    +          return valueToExpose;
    +        }
    +      });
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Note that reporters will turn the exposed object into a `String`, which means that a meaningful `toString()` implementation is required.
    +
    +#### Histogram
    +
    +A `Histogram` measures the distribution of long values.
    +You can register one by calling `histogram(String name, Histogram histogram)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.histogram = getRuntimeContext()
    +      .getMetricGroup()
    +      .histogram("myHistogram", new MyHistogram());
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink does not provide a default implementation for `Histogram`, but offers a {% gh_link flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/DropwizardHistogramWrapper.java "Wrapper" %} that allows usage of Codahale/DropWizard histograms.
    +To use this wrapper add the following dependency in your `pom.xml`:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-dropwizard</artifactId>
    +      <version>{{site.version}}</version>
    +</dependency>
    +{% endhighlight %}
    +
    +You can then register a Codahale/DropWizard histogram like this:
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    com.codahale.metrics.Histogram histogram =
    +      new com.codahale.metrics.Histogram(new SlidingWindowReservoir(500));
    +
    +    this.histogram = getRuntimeContext()
    +      .getMetricGroup()
    +      .histogram("myHistogram", new DropWizardHistogramWrapper(histogram));
    +  }
    +}
    +{% endhighlight %}
    +
    +## Scope
    +
    +Every metric is assigned an identifier under which it will be reported that is based on 3 components: the user-provided name when registering the metric, an optional user-defined scope and a system-provided scope.
    +For example, if `A.B` is the sytem scope, `C.D` the user scope and `E` the name, then the identifier for the metric will be `A.B.C.D.E`.
    +
    +### User Scope
    +
    +You can define a user scope by calling either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext()
    +  .getMetricGroup()
    +  .addGroup("MyMetrics")
    +  .counter("myCounter");
    +
    +{% endhighlight %}
    +
    +### System Scope
    +
    +The system scope contains context information about the metric, for example in which task it was registered or what job that task belongs to.
    +
    +Which context information should be included can be configured by setting the following keys in `conf/flink-conf.yaml`.
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "&lt;task_id&gt;") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: &lt;host&gt;.jobmanager
    +  - Applied to all metrics that were scoped to a job manager.
    +- `metrics.scope.jm.job`
    +  - Default: &lt;host&gt;.jobmanager.&lt;job_name&gt;
    +  - Applied to all metrics that were scoped to a job manager and job.
    +- `metrics.scope.tm`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;
    +  - Applied to all metrics that were scoped to a task manager.
    +- `metrics.scope.tm.job`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;
    +  - Applied to all metrics that were scoped to a task manager and job.
    +- `metrics.scope.tm.task`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;.&lt;task_name&gt;.&lt;subtask_index&gt;
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;.&lt;operator_name&gt;.&lt;subtask_index&gt;
    +  - Applied to all metrics that were scoped to an operator.
    +
    +There are no restrictions on the number or order of variables. Variables are case sensitive.
    +
    +The default scope for operator metrics will result in an identifier akin to `localhost.taskmanager.1234.MyJob.MyOperator.0.MyMetric`
    +
    +If you also want to include the task name but omit the task manager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: <host>.<job_name>.<task_name>.<operator_name>.<subtask_index>`
    +
    +This could create the identifier `localhost.MyJob.MySource_->_MyOperator.MyOperator.0.MyMetric`.
    +
    +Note that for this format string an identifier clash can occur should the same job be run multiple times concurrently, which can lead to inconsistent metric data.
    +As such it is advised to either use format strings that provide a certain degree of uniqueness by including IDs (e.g &lt;job_id&gt;)
    +or by assigning unique names to jobs and operators.
    +
    +### List of all Variables
    +
    +- JobManager: &lt;host&gt;
    +- TaskManager: &lt;host&gt;, &lt;tm_id&gt;
    +- Job: &lt;job_id&gt;, &lt;job_name&gt;
    +- Task: &lt;task_id&gt;, &lt;task_name&gt;, &lt;task_attempt_id&gt;, &lt;task_attempt_num&gt;, &lt;subtask_index&gt;
    +- Operator: &lt;operator_name&gt;, &lt;subtask_index&gt;
    +
    +## Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS
    +
    +You can write your own `Reporter` by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the `Scheduled` interface as well.
    +
    +By default Flink uses JMX to expose metrics.
    +All non-JMXReporters are not part of the distribution and have to be added [manually]({{site.baseurl}}/apis/cluster_execution.html#linking-with-modules-not-contained-in-the-binary-distribution).
    +
    +The following sections list the supported reporters.
    +
    +### JMX
    +
    +The port for JMX can be configured by setting the `metrics.jmx.port` key. This parameter expects either a single port
    +or a port range, with the default being 9010-9025. The used port is shown in the relevant job or task manager log.
    +
    +### Ganglia (org.apache.flink.metrics.ganglia.GangliaReporter)
    +Dependency:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-ganglia</artifactId>
    +      <version>{{site.version}}</version>
    +</dependency>
    +{% endhighlight %}
    +
    +Parameters:
    +
    +- `host` - the ganglia server host
    +- `port` - the ganglia server port
    +- `tmax` - soft limit for how long an old metric should be retained
    +- `dmax` - hard limit for how long an old metric should be retained
    +- `ttl` - time-to-live for transmitted UDP packets
    +- `addressingMode` - UDP addressing mode to use (UNICAST/MULTICAST)
    +
    +### Graphite (org.apache.flink.metrics.graphite.GraphiteReporter)
    +Dependency:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-graphite</artifactId>
    +      <version>{{site.version}}</version>
    +</dependency>
    --- End diff --
    
    We should document that the user has to put the `flink-metrics-graphite-VERSION-jar-with-dependencies.jar` in the `lib` folder or make it differently accessible to the cluster when starting it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68987851
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    +      @Override
    +      public Integer getValue() {
    +        return valueToExpose;
    +      }});
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Histogram
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.histogram = getRuntimeContext().getMetricGroup().histogram("myHistogram", new MyHistogram());
    +    ...
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +    ...
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +This wrapper is contained in the `flink-metrics-dropwizard` module.
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represents the entities it is tied to. By default a metric that is registered in a user function will be scoped to the operator in which the function runs, the task/job it belongs to and the taskManager/host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter()` call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +  - Applied to all metrics that were scoped to a jobmanager.
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +  - Applied to all metrics that were scoped to a jobmanager and job.
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +  - Applied to all metrics that were scoped to a taskmanager.
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +  - Applied to all metrics that were scoped to a taskmanager and job.
    +- `metrics.scope.tm.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +  - Applied to all metrics that were scoped to an operator.
    +
    +Note that for metrics for which multiple formats may apply (like jm and jm.job) the most specific format takes precedence,
    +in this case jm.job.
    +
    +The hierarchical orders are as follows:
    +
    +jm < jm.job 
    +
    +tm < tm.job < tm.task < tm.operator
    +
    +This hierarchy also defines which variables may be accessed. The `tm.operator` format may contain variables for jobs, whereas `tm.job` may not contain
    +variables for operators. There is no restriction on order of variables.
    +
    +There is no restriction on the order of variables.
    +
    +The default scope for operator metrics will result in a metric name akin to `localhost.taskmanager.1234.MyJob.MyOperator.0`
    +
    +If you also want to include the task name, but omit the taskmanager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: \<host>.\<job_name>.\<task_name>.\<operator_name>.\<subtask_index>`
    --- End diff --
    
    Without the `\` they are interpreted as tags and not as text.
    
    I used the build_docs.sh script to preview the docs, and the backslashes did not appear.
    
    But I could (and probably should) replace all the < and > with `&lt;` and `&gt;` respectively.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by uce <gi...@git.apache.org>.
Github user uce commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69162793
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    --- End diff --
    
    `getmetricGroup` with lower letter `m`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2158: [FLINK-4116] Metrics documentation

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on the issue:

    https://github.com/apache/flink/pull/2158
  
    I've made the following changes:
    - rewritte scope section
    - removed ... from examples
    - FIX ALL THE TYPOS
    - better line breaking
    - added dependency&example for DWHWrapper
    - replaced hard-coded dependency versions


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68563199
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter() call. The order is always \<system_scope>\<user_scope>\<name>.
    --- End diff --
    
    counter has only one backtick. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by uce <gi...@git.apache.org>.
Github user uce commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69166767
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    +      @Override
    +      public Integer getValue() {
    +        return valueToExpose;
    +      }});
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Histogram
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.histogram = getRuntimeContext().getMetricGroup().histogram("myHistogram", new MyHistogram());
    +    ...
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +    ...
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +This wrapper is contained in the `flink-metrics-dropwizard` module.
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represents the entities it is tied to. By default a metric that is registered in a user function will be scoped to the operator in which the function runs, the task/job it belongs to and the taskManager/host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter()` call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +  - Applied to all metrics that were scoped to a jobmanager.
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +  - Applied to all metrics that were scoped to a jobmanager and job.
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +  - Applied to all metrics that were scoped to a taskmanager.
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +  - Applied to all metrics that were scoped to a taskmanager and job.
    +- `metrics.scope.tm.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +  - Applied to all metrics that were scoped to an operator.
    +
    +Note that for metrics for which multiple formats may apply (like jm and jm.job) the most specific format takes precedence,
    +in this case jm.job.
    +
    +The hierarchical orders are as follows:
    +
    +jm < jm.job 
    +
    +tm < tm.job < tm.task < tm.operator
    +
    +This hierarchy also defines which variables may be accessed. The `tm.operator` format may contain variables for jobs, whereas `tm.job` may not contain
    +variables for operators. There is no restriction on order of variables.
    +
    +There is no restriction on the order of variables.
    +
    +The default scope for operator metrics will result in a metric name akin to `localhost.taskmanager.1234.MyJob.MyOperator.0`
    +
    +If you also want to include the task name, but omit the taskmanager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: \<host>.\<job_name>.\<task_name>.\<operator_name>.\<subtask_index>`
    +
    +This could create the name `localhost.MyJob.MySource_->_MyOperator.MyOperator.0`.
    +
    +The following is a list of all variables available:
    --- End diff --
    
    I think it makes sense to make this a subsection so that we can directly link to it etc.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68563758
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter() call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +- `metrics.scope.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +- `metrics.scope.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +There is no restriction on the order of variables.
    +
    +Which format is applied to a metric depends on which entities it was scoped to.
    +The `metrics.scope.operator` format will be applied to all metrics that were scoped to an Operator.
    +The `metrics.scope.tm.job` format will be applied to all metrics that were only scoped to a Job and a TaskManager.
    --- End diff --
    
    Maybe it would be good to add the explanation of the different scopes directly to the enumeration of the config keys instead of listing them in this paragraph.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2158: [FLINK-4116] Metrics documentation

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on the issue:

    https://github.com/apache/flink/pull/2158
  
    I've addressed all comments, except the config page.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69270577
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    +      @Override
    +      public Integer getValue() {
    +        return valueToExpose;
    +      }});
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Histogram
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.histogram = getRuntimeContext().getMetricGroup().histogram("myHistogram", new MyHistogram());
    +    ...
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +    ...
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +This wrapper is contained in the `flink-metrics-dropwizard` module.
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represents the entities it is tied to. By default a metric that is registered in a user function will be scoped to the operator in which the function runs, the task/job it belongs to and the taskManager/host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter()` call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +  - Applied to all metrics that were scoped to a jobmanager.
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +  - Applied to all metrics that were scoped to a jobmanager and job.
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +  - Applied to all metrics that were scoped to a taskmanager.
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +  - Applied to all metrics that were scoped to a taskmanager and job.
    +- `metrics.scope.tm.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +  - Applied to all metrics that were scoped to an operator.
    +
    +Note that for metrics for which multiple formats may apply (like jm and jm.job) the most specific format takes precedence,
    +in this case jm.job.
    +
    +The hierarchical orders are as follows:
    +
    +jm < jm.job 
    +
    +tm < tm.job < tm.task < tm.operator
    +
    +This hierarchy also defines which variables may be accessed. The `tm.operator` format may contain variables for jobs, whereas `tm.job` may not contain
    +variables for operators. There is no restriction on order of variables.
    +
    +There is no restriction on the order of variables.
    +
    +The default scope for operator metrics will result in a metric name akin to `localhost.taskmanager.1234.MyJob.MyOperator.0`
    +
    +If you also want to include the task name, but omit the taskmanager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: \<host>.\<job_name>.\<task_name>.\<operator_name>.\<subtask_index>`
    --- End diff --
    
    oh, in this case we don't need them since they are in backticks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68931051
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter() call. The order is always \<system_scope>\<user_scope>\<name>.
    --- End diff --
    
    Hmm I think you're right, since the periods won't probably be rendered differently. But you could put the whole name in quotation marks. Then it should be clear.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68561851
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    --- End diff --
    
    `getRuntimeContext().getMetricGroup` should be enclosed by back ticks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by uce <gi...@git.apache.org>.
Github user uce commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69162949
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    +      @Override
    +      public Integer getValue() {
    +        return valueToExpose;
    +      }});
    +    ...
    --- End diff --
    
    My personal taste, but I think it's better to just remove this (and other) three dots from the examples. Makes copy pasting easier and it should be clear from context that other code can be added as needed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68564324
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter() call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +- `metrics.scope.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +- `metrics.scope.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +There is no restriction on the order of variables.
    +
    +Which format is applied to a metric depends on which entities it was scoped to.
    +The `metrics.scope.operator` format will be applied to all metrics that were scoped to an Operator.
    +The `metrics.scope.tm.job` format will be applied to all metrics that were only scoped to a Job and a TaskManager.
    +
    +This allows you to define explicitly how metrics on every level should be reported.
    +
    +### Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in the `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter)
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010)
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS)
    +
    +By default Flink uses JMX to expose metrics.
    +
    +The port for JMX can be configured by setting the `metrics.jmx.port` key. This parameter expects either a single port
    +or a port range, with the default being 9010-9025. Which port is used in the end is shown in the relevant Job-/TaskManager log.
    +
    +All non-JMXReporters are not part of the distribution and have to be added to the classpath manually. (usually by putting the jar into /lib)
    +
    +You can write your own Reporter by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the Scheduled interface as well.
    +
    +Flink supports the following systems with their respective configuration parameters:
    +
    +#### Ganglia (org.apcahe.flink.metrics.ganglia.GangliaReporter)
    +- `host`
    +- `port`
    +- `dmax`
    +- `tmax`
    +- `ttl`
    +- `addressingMode` (UNICAST/MULTICAST)
    +
    +#### Graphite (org.apache.flink.metrics.graphite.GraphiteReporter)
    +- `host`
    +- `port`
    +
    +#### StatsD (org.apache.flink.metrics.statsd.StatsDReporter)
    +- `host`
    +- `port`
    +
    +### System metrics
    +
    +Flink exposes the following system metrics:
    +
    +<table class="table table-bordered">
    +  <thead>
    +    <tr>
    +      <th class="text-left" style="width: 20%">Scope</th>
    +      <th class="text-center">Metrics</th>
    +    </tr>
    +  </thead>
    +
    +  <tbody>
    +    <tr>
    +      <td><strong>JobManager</strong></td>
    +      <td>
    +      </td>
    +    </tr>
    +    <tr>
    +      <td><strong>TaskManager</strong></td>
    +      <td>
    +        <p>Status.JVM.ClassLoader.ClassesLoaded</p>
    +        <p>Status.JVM.ClassLoader.ClassesUnloaded</p>
    +        <p></p>
    +        <p>Status.JVM.GargabeCollector.&lt;garbageCollector&gt;.Count</p>
    +        <p>Status.JVM.GargabeCollector.&lt;garbageCollector&gt;.Time</p>
    +        <p></p>
    +        <p>Status.JVM.Memory.Heap.Used</p>
    +        <p>Status.JVM.Memory.Heap.Committed</p>
    +        <p>Status.JVM.Memory.Heap.Max</p>
    +        <p>Status.JVM.Memory.NonHeap.Used</p>
    +        <p>Status.JVM.Memory.NonHeap.Committed</p>
    +        <p>Status.JVM.Memory.NonHeap.Max</p>
    +        <p>Status.JVM.Memory.Direct.Count</p>
    +        <p>Status.JVM.Memory.Direct.MemoryUsed</p>
    +        <p>Status.JVM.Memory.Direct.TotalCapacity</p>
    +        <p>Status.JVM.Memory.Mapped.Count</p>
    +        <p>Status.JVM.Memory.Mapped.MemoryUsed</p>
    +        <p>Status.JVM.Memory.Mapped.TotalCapacity</p>
    +        <p></p>
    +        <p>Status.JVM.Threads.Count</p>
    +        <p></p>
    +        <p>Status.JVM.CPU.Load</p>
    +        <p>Status.JVM.CPU.Time</p>
    +      </td>
    +    </tr>
    +    <tr>
    +      <td><strong>Job</strong></td>
    +      <td>
    +      </td>
    +    </tr>
    +    <tr>
    +      <td><strong>Task</strong></td>
    +      <td>
    +        <p>currentLowWatermark</p>
    +        <p>lastCheckpointSize</p>
    +        <p>numBytesInLocal</p>
    +        <p>numBytesInRemote</p>
    +        <p>numBytesOut</p>
    +      </td>
    +    </tr>
    +    <tr>
    +      <td><strong>Operator</strong></td>
    +      <td>
    +        <p>numRecordsIn</p>
    +        <p>numRecordsOut</p>
    +        <p>numSplitsProcessed</p>
    +      </td>
    +    </tr>
    +  </tbody>
    +</table>
    --- End diff --
    
    Explanation of metrics could be helpful.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68562298
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    --- End diff --
    
    I've never seen the notation `inc([long n])` in the Flink documentation. Maybe it's better to specify the parameterless and the parameter variant explicitly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by uce <gi...@git.apache.org>.
Github user uce commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69163472
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    +      @Override
    +      public Integer getValue() {
    +        return valueToExpose;
    +      }});
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Histogram
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    --- End diff --
    
    missing code formating for `histogram(...)` (via wrap in backtick)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68571093
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter() call. The order is always \<system_scope>\<user_scope>\<name>.
    --- End diff --
    
    kind of. i guess technically the delimiter is part of every group. I wanted to avoid adding `.` between the scope parts as it may lead to the misunderstanding that there are always only 2 dots in the name.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68564218
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter() call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +- `metrics.scope.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +- `metrics.scope.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +There is no restriction on the order of variables.
    +
    +Which format is applied to a metric depends on which entities it was scoped to.
    +The `metrics.scope.operator` format will be applied to all metrics that were scoped to an Operator.
    +The `metrics.scope.tm.job` format will be applied to all metrics that were only scoped to a Job and a TaskManager.
    +
    +This allows you to define explicitly how metrics on every level should be reported.
    +
    +### Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in the `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter)
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010)
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS)
    +
    +By default Flink uses JMX to expose metrics.
    +
    +The port for JMX can be configured by setting the `metrics.jmx.port` key. This parameter expects either a single port
    +or a port range, with the default being 9010-9025. Which port is used in the end is shown in the relevant Job-/TaskManager log.
    +
    +All non-JMXReporters are not part of the distribution and have to be added to the classpath manually. (usually by putting the jar into /lib)
    +
    +You can write your own Reporter by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the Scheduled interface as well.
    +
    +Flink supports the following systems with their respective configuration parameters:
    +
    +#### Ganglia (org.apcahe.flink.metrics.ganglia.GangliaReporter)
    +- `host`
    +- `port`
    +- `dmax`
    +- `tmax`
    +- `ttl`
    +- `addressingMode` (UNICAST/MULTICAST)
    +
    +#### Graphite (org.apache.flink.metrics.graphite.GraphiteReporter)
    +- `host`
    +- `port`
    --- End diff --
    
    Explanation of parameters


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68564061
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter() call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +- `metrics.scope.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +- `metrics.scope.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +There is no restriction on the order of variables.
    +
    +Which format is applied to a metric depends on which entities it was scoped to.
    +The `metrics.scope.operator` format will be applied to all metrics that were scoped to an Operator.
    +The `metrics.scope.tm.job` format will be applied to all metrics that were only scoped to a Job and a TaskManager.
    +
    +This allows you to define explicitly how metrics on every level should be reported.
    +
    +### Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in the `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter)
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010)
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS)
    +
    +By default Flink uses JMX to expose metrics.
    +
    +The port for JMX can be configured by setting the `metrics.jmx.port` key. This parameter expects either a single port
    +or a port range, with the default being 9010-9025. Which port is used in the end is shown in the relevant Job-/TaskManager log.
    +
    +All non-JMXReporters are not part of the distribution and have to be added to the classpath manually. (usually by putting the jar into /lib)
    +
    +You can write your own Reporter by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the Scheduled interface as well.
    --- End diff --
    
    Reporter and Scheduled should be put into backticks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by uce <gi...@git.apache.org>.
Github user uce commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69165618
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    +      @Override
    +      public Integer getValue() {
    +        return valueToExpose;
    +      }});
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Histogram
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.histogram = getRuntimeContext().getMetricGroup().histogram("myHistogram", new MyHistogram());
    +    ...
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +    ...
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +This wrapper is contained in the `flink-metrics-dropwizard` module.
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represents the entities it is tied to. By default a metric that is registered in a user function will be scoped to the operator in which the function runs, the task/job it belongs to and the taskManager/host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter()` call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +  - Applied to all metrics that were scoped to a jobmanager.
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +  - Applied to all metrics that were scoped to a jobmanager and job.
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +  - Applied to all metrics that were scoped to a taskmanager.
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +  - Applied to all metrics that were scoped to a taskmanager and job.
    +- `metrics.scope.tm.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +  - Applied to all metrics that were scoped to an operator.
    +
    +Note that for metrics for which multiple formats may apply (like jm and jm.job) the most specific format takes precedence,
    +in this case jm.job.
    +
    +The hierarchical orders are as follows:
    +
    +jm < jm.job 
    +
    +tm < tm.job < tm.task < tm.operator
    +
    +This hierarchy also defines which variables may be accessed. The `tm.operator` format may contain variables for jobs, whereas `tm.job` may not contain
    +variables for operators. There is no restriction on order of variables.
    +
    +There is no restriction on the order of variables.
    +
    +The default scope for operator metrics will result in a metric name akin to `localhost.taskmanager.1234.MyJob.MyOperator.0`
    +
    +If you also want to include the task name, but omit the taskmanager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: \<host>.\<job_name>.\<task_name>.\<operator_name>.\<subtask_index>`
    --- End diff --
    
    Same for me in Safari


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68590134
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter() call. The order is always \<system_scope>\<user_scope>\<name>.
    --- End diff --
    
    We could put it into backticks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68563783
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter() call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +- `metrics.scope.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +- `metrics.scope.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +There is no restriction on the order of variables.
    +
    +Which format is applied to a metric depends on which entities it was scoped to.
    +The `metrics.scope.operator` format will be applied to all metrics that were scoped to an Operator.
    +The `metrics.scope.tm.job` format will be applied to all metrics that were only scoped to a Job and a TaskManager.
    +
    +This allows you to define explicitly how metrics on every level should be reported.
    --- End diff --
    
    Giving a small example could be helpful.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by uce <gi...@git.apache.org>.
Github user uce commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69165992
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    +      @Override
    +      public Integer getValue() {
    +        return valueToExpose;
    +      }});
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Histogram
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.histogram = getRuntimeContext().getMetricGroup().histogram("myHistogram", new MyHistogram());
    +    ...
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +    ...
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +This wrapper is contained in the `flink-metrics-dropwizard` module.
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represents the entities it is tied to. By default a metric that is registered in a user function will be scoped to the operator in which the function runs, the task/job it belongs to and the taskManager/host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter()` call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +  - Applied to all metrics that were scoped to a jobmanager.
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +  - Applied to all metrics that were scoped to a jobmanager and job.
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +  - Applied to all metrics that were scoped to a taskmanager.
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +  - Applied to all metrics that were scoped to a taskmanager and job.
    +- `metrics.scope.tm.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +  - Applied to all metrics that were scoped to an operator.
    +
    +Note that for metrics for which multiple formats may apply (like jm and jm.job) the most specific format takes precedence,
    +in this case jm.job.
    +
    +The hierarchical orders are as follows:
    +
    +jm < jm.job 
    +
    +tm < tm.job < tm.task < tm.operator
    +
    +This hierarchy also defines which variables may be accessed. The `tm.operator` format may contain variables for jobs, whereas `tm.job` may not contain
    +variables for operators. There is no restriction on order of variables.
    +
    +There is no restriction on the order of variables.
    +
    +The default scope for operator metrics will result in a metric name akin to `localhost.taskmanager.1234.MyJob.MyOperator.0`
    +
    +If you also want to include the task name, but omit the taskmanager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: \<host>.\<job_name>.\<task_name>.\<operator_name>.\<subtask_index>`
    +
    +This could create the name `localhost.MyJob.MySource_->_MyOperator.MyOperator.0`.
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +### Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in the `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS
    +
    +You can write your own `Reporter` by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the `Scheduled` interface as well.
    +
    +By default Flink uses JMX to expose metrics.
    +All non-JMXReporters are not part of the distribution and have to be added to the classpath manually, either by putting the jar into /lib
    --- End diff --
    
    I think we also link to an explanation in another docs page for this


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by uce <gi...@git.apache.org>.
Github user uce commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69162434
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    --- End diff --
    
    typo: not => no


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68904562
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a `Counter` like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own `Counter` implementation along with the name.
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represents the entities it is tied to. By default a metric that is registered in a user function will be scoped to the operator in which the function runs, the task/job it belongs to and the taskManager/host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter()` call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +- `metrics.scope.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +- `metrics.scope.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +There is no restriction on the order of variables.
    +
    +Which format is applied to a metric depends on which entities it was scoped to.
    +The `metrics.scope.operator` format will be applied to all metrics that were scoped to an operator.
    +The `metrics.scope.tm.job` format will be applied to all metrics that were only scoped to a job and a taskmanager.
    +
    +This allows you to define explicitly how metrics on every level should be reported.
    +
    +### Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in the `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter)
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010)
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS)
    +
    +By default Flink uses JMX to expose metrics.
    +
    +The port for JMX can be configured by setting the `metrics.jmx.port` key. This parameter expects either a single port
    +or a port range, with the default being 9010-9025. Which port is used in the end is shown in the relevant Job-/TaskManager log.
    +
    +All non-JMXReporters are not part of the distribution and have to be added to the classpath manually, usually by putting the jar into /lib.
    +
    +You can write your own Reporter by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the Scheduled interface as well.
    +
    +Flink supports the following systems with their respective configuration parameters:
    +
    +#### Ganglia (org.apcahe.flink.metrics.ganglia.GangliaReporter)
    --- End diff --
    
    typo in apcahe


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by uce <gi...@git.apache.org>.
Github user uce commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69162546
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    --- End diff --
    
    btw does it not need to be serializable?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by uce <gi...@git.apache.org>.
Github user uce commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69165738
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    +      @Override
    +      public Integer getValue() {
    +        return valueToExpose;
    +      }});
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Histogram
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.histogram = getRuntimeContext().getMetricGroup().histogram("myHistogram", new MyHistogram());
    +    ...
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +    ...
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +This wrapper is contained in the `flink-metrics-dropwizard` module.
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represents the entities it is tied to. By default a metric that is registered in a user function will be scoped to the operator in which the function runs, the task/job it belongs to and the taskManager/host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter()` call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +  - Applied to all metrics that were scoped to a jobmanager.
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +  - Applied to all metrics that were scoped to a jobmanager and job.
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +  - Applied to all metrics that were scoped to a taskmanager.
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +  - Applied to all metrics that were scoped to a taskmanager and job.
    +- `metrics.scope.tm.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +  - Applied to all metrics that were scoped to an operator.
    +
    +Note that for metrics for which multiple formats may apply (like jm and jm.job) the most specific format takes precedence,
    +in this case jm.job.
    +
    +The hierarchical orders are as follows:
    +
    +jm < jm.job 
    +
    +tm < tm.job < tm.task < tm.operator
    +
    +This hierarchy also defines which variables may be accessed. The `tm.operator` format may contain variables for jobs, whereas `tm.job` may not contain
    +variables for operators. There is no restriction on order of variables.
    +
    +There is no restriction on the order of variables.
    +
    +The default scope for operator metrics will result in a metric name akin to `localhost.taskmanager.1234.MyJob.MyOperator.0`
    +
    +If you also want to include the task name, but omit the taskmanager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: \<host>.\<job_name>.\<task_name>.\<operator_name>.\<subtask_index>`
    +
    +This could create the name `localhost.MyJob.MySource_->_MyOperator.MyOperator.0`.
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +### Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in the `conf/flink-conf.yaml`.
    --- End diff --
    
    Either `in conf/flink-conf.yaml` or `in the ... file`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68562169
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    --- End diff --
    
    backticks or writing counters, gauges and histograms without a capital letter.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68564101
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter() call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +- `metrics.scope.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +- `metrics.scope.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +There is no restriction on the order of variables.
    +
    +Which format is applied to a metric depends on which entities it was scoped to.
    +The `metrics.scope.operator` format will be applied to all metrics that were scoped to an Operator.
    +The `metrics.scope.tm.job` format will be applied to all metrics that were only scoped to a Job and a TaskManager.
    +
    +This allows you to define explicitly how metrics on every level should be reported.
    +
    +### Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in the `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter)
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010)
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS)
    +
    +By default Flink uses JMX to expose metrics.
    +
    +The port for JMX can be configured by setting the `metrics.jmx.port` key. This parameter expects either a single port
    +or a port range, with the default being 9010-9025. Which port is used in the end is shown in the relevant Job-/TaskManager log.
    +
    +All non-JMXReporters are not part of the distribution and have to be added to the classpath manually. (usually by putting the jar into /lib)
    +
    +You can write your own Reporter by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the Scheduled interface as well.
    --- End diff --
    
    Maybe we should explain what `Scheduled` indicates.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69913430
  
    --- Diff: docs/apis/metrics.md ---
    @@ -0,0 +1,441 @@
    +---
    +title: "Metrics"
    +# Top-level navigation
    +top-nav-group: apis
    +top-nav-pos: 13
    +top-nav-title: "Metrics"
    +---
    +<!--
    +Licensed to the Apache Software Foundation (ASF) under one
    +or more contributor license agreements.  See the NOTICE file
    +distributed with this work for additional information
    +regarding copyright ownership.  The ASF licenses this file
    +to you under the Apache License, Version 2.0 (the
    +"License"); you may not use this file except in compliance
    +with the License.  You may obtain a copy of the License at
    +
    +  http://www.apache.org/licenses/LICENSE-2.0
    +
    +Unless required by applicable law or agreed to in writing,
    +software distributed under the License is distributed on an
    +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    +KIND, either express or implied.  See the License for the
    +specific language governing permissions and limitations
    +under the License.
    +-->
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +* This will be replaced by the TOC
    +{:toc}
    +
    +## Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.counter = getRuntimeContext()
    +      .getMetricGroup()
    +      .counter("myCounter");
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    this.counter.inc();
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.counter = getRuntimeContext()
    +      .getMetricGroup()
    +      .counter("myCustomCounter", new CustomCounter());
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is no restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    getRuntimeContext()
    +      .getMetricGroup()
    +      .gauge("MyGauge", new Gauge<Integer>() {
    +        @Override
    +        public Integer getValue() {
    +          return valueToExpose;
    +        }
    +      });
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Note that reporters will turn the exposed object into a `String`, which means that a meaningful `toString()` implementation is required.
    +
    +#### Histogram
    +
    +A `Histogram` measures the distribution of long values.
    +You can register one by calling `histogram(String name, Histogram histogram)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.histogram = getRuntimeContext()
    +      .getMetricGroup()
    +      .histogram("myHistogram", new MyHistogram());
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink does not provide a default implementation for `Histogram`, but offers a {% gh_link flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/DropwizardHistogramWrapper.java "Wrapper" %} that allows usage of Codahale/DropWizard histograms.
    +To use this wrapper add the following dependency in your `pom.xml`:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-dropwizard</artifactId>
    +      <version>{{site.version}}</version>
    +</dependency>
    +{% endhighlight %}
    +
    +You can then register a Codahale/DropWizard histogram like this:
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    com.codahale.metrics.Histogram histogram =
    +      new com.codahale.metrics.Histogram(new SlidingWindowReservoir(500));
    +
    +    this.histogram = getRuntimeContext()
    +      .getMetricGroup()
    +      .histogram("myHistogram", new DropWizardHistogramWrapper(histogram));
    +  }
    +}
    +{% endhighlight %}
    +
    +## Scope
    +
    +Every metric is assigned an identifier under which it will be reported that is based on 3 components: the user-provided name when registering the metric, an optional user-defined scope and a system-provided scope.
    +For example, if `A.B` is the sytem scope, `C.D` the user scope and `E` the name, then the identifier for the metric will be `A.B.C.D.E`.
    +
    +### User Scope
    +
    +You can define a user scope by calling either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext()
    +  .getMetricGroup()
    +  .addGroup("MyMetrics")
    +  .counter("myCounter");
    +
    +{% endhighlight %}
    +
    +### System Scope
    +
    +The system scope contains context information about the metric, for example in which task it was registered or what job that task belongs to.
    +
    +Which context information should be included can be configured by setting the following keys in `conf/flink-conf.yaml`.
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "&lt;task_id&gt;") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: &lt;host&gt;.jobmanager
    +  - Applied to all metrics that were scoped to a job manager.
    +- `metrics.scope.jm.job`
    +  - Default: &lt;host&gt;.jobmanager.&lt;job_name&gt;
    +  - Applied to all metrics that were scoped to a job manager and job.
    +- `metrics.scope.tm`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;
    +  - Applied to all metrics that were scoped to a task manager.
    +- `metrics.scope.tm.job`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;
    +  - Applied to all metrics that were scoped to a task manager and job.
    +- `metrics.scope.tm.task`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;.&lt;task_name&gt;.&lt;subtask_index&gt;
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;.&lt;operator_name&gt;.&lt;subtask_index&gt;
    +  - Applied to all metrics that were scoped to an operator.
    +
    +There are no restrictions on the number or order of variables. Variables are case sensitive.
    +
    +The default scope for operator metrics will result in an identifier akin to `localhost.taskmanager.1234.MyJob.MyOperator.0.MyMetric`
    +
    +If you also want to include the task name but omit the task manager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: <host>.<job_name>.<task_name>.<operator_name>.<subtask_index>`
    +
    +This could create the identifier `localhost.MyJob.MySource_->_MyOperator.MyOperator.0.MyMetric`.
    +
    +Note that for this format string an identifier clash can occur should the same job be run multiple times concurrently, which can lead to inconsistent metric data.
    +As such it is advised to either use format strings that provide a certain degree of uniqueness by including IDs (e.g &lt;job_id&gt;)
    +or by assigning unique names to jobs and operators.
    +
    +### List of all Variables
    +
    +- JobManager: &lt;host&gt;
    +- TaskManager: &lt;host&gt;, &lt;tm_id&gt;
    +- Job: &lt;job_id&gt;, &lt;job_name&gt;
    +- Task: &lt;task_id&gt;, &lt;task_name&gt;, &lt;task_attempt_id&gt;, &lt;task_attempt_num&gt;, &lt;subtask_index&gt;
    +- Operator: &lt;operator_name&gt;, &lt;subtask_index&gt;
    +
    +## Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS
    +
    +You can write your own `Reporter` by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the `Scheduled` interface as well.
    +
    +By default Flink uses JMX to expose metrics.
    +All non-JMXReporters are not part of the distribution and have to be added [manually]({{site.baseurl}}/apis/cluster_execution.html#linking-with-modules-not-contained-in-the-binary-distribution).
    +
    +The following sections list the supported reporters.
    +
    +### JMX
    +
    +The port for JMX can be configured by setting the `metrics.jmx.port` key. This parameter expects either a single port
    +or a port range, with the default being 9010-9025. The used port is shown in the relevant job or task manager log.
    +
    +### Ganglia (org.apache.flink.metrics.ganglia.GangliaReporter)
    +Dependency:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-ganglia</artifactId>
    +      <version>{{site.version}}</version>
    +</dependency>
    +{% endhighlight %}
    +
    +Parameters:
    +
    +- `host` - the ganglia server host
    +- `port` - the ganglia server port
    +- `tmax` - soft limit for how long an old metric should be retained
    +- `dmax` - hard limit for how long an old metric should be retained
    +- `ttl` - time-to-live for transmitted UDP packets
    +- `addressingMode` - UDP addressing mode to use (UNICAST/MULTICAST)
    +
    +### Graphite (org.apache.flink.metrics.graphite.GraphiteReporter)
    +Dependency:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-graphite</artifactId>
    +      <version>{{site.version}}</version>
    +</dependency>
    --- End diff --
    
    The same probably applies to the `GangliaReporter`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68561918
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    --- End diff --
    
    `MetricGroup` should have backticks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on the issue:

    https://github.com/apache/flink/pull/2158
  
    Good first draft of the metrics documentation @zentol. I think the documentation should be a bit more verbose containing more explanations and code examples. This will help the user to quickly get started and understanding the metrics system.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68564233
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter() call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +- `metrics.scope.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +- `metrics.scope.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +There is no restriction on the order of variables.
    +
    +Which format is applied to a metric depends on which entities it was scoped to.
    +The `metrics.scope.operator` format will be applied to all metrics that were scoped to an Operator.
    +The `metrics.scope.tm.job` format will be applied to all metrics that were only scoped to a Job and a TaskManager.
    +
    +This allows you to define explicitly how metrics on every level should be reported.
    +
    +### Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in the `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter)
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010)
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS)
    +
    +By default Flink uses JMX to expose metrics.
    +
    +The port for JMX can be configured by setting the `metrics.jmx.port` key. This parameter expects either a single port
    +or a port range, with the default being 9010-9025. Which port is used in the end is shown in the relevant Job-/TaskManager log.
    +
    +All non-JMXReporters are not part of the distribution and have to be added to the classpath manually. (usually by putting the jar into /lib)
    +
    +You can write your own Reporter by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the Scheduled interface as well.
    +
    +Flink supports the following systems with their respective configuration parameters:
    +
    +#### Ganglia (org.apcahe.flink.metrics.ganglia.GangliaReporter)
    +- `host`
    +- `port`
    +- `dmax`
    +- `tmax`
    +- `ttl`
    +- `addressingMode` (UNICAST/MULTICAST)
    +
    +#### Graphite (org.apache.flink.metrics.graphite.GraphiteReporter)
    +- `host`
    +- `port`
    +
    +#### StatsD (org.apache.flink.metrics.statsd.StatsDReporter)
    +- `host`
    +- `port`
    --- End diff --
    
    Explanation of parameters


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68562924
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    --- End diff --
    
    which represents


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68564011
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter() call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +- `metrics.scope.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +- `metrics.scope.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +There is no restriction on the order of variables.
    +
    +Which format is applied to a metric depends on which entities it was scoped to.
    +The `metrics.scope.operator` format will be applied to all metrics that were scoped to an Operator.
    +The `metrics.scope.tm.job` format will be applied to all metrics that were only scoped to a Job and a TaskManager.
    +
    +This allows you to define explicitly how metrics on every level should be reported.
    +
    +### Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in the `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter)
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010)
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS)
    +
    +By default Flink uses JMX to expose metrics.
    +
    +The port for JMX can be configured by setting the `metrics.jmx.port` key. This parameter expects either a single port
    +or a port range, with the default being 9010-9025. Which port is used in the end is shown in the relevant Job-/TaskManager log.
    +
    +All non-JMXReporters are not part of the distribution and have to be added to the classpath manually. (usually by putting the jar into /lib)
    --- End diff --
    
    parenthesis starts after the period


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68562986
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    --- End diff --
    
    operator with small letter.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68563569
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter() call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +- `metrics.scope.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +- `metrics.scope.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +There is no restriction on the order of variables.
    +
    +Which format is applied to a metric depends on which entities it was scoped to.
    +The `metrics.scope.operator` format will be applied to all metrics that were scoped to an Operator.
    --- End diff --
    
    operator with small letter 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by uce <gi...@git.apache.org>.
Github user uce commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69162153
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    --- End diff --
    
    Orthogonal discussion, but the name `counter` to create a counter (and others respectively) is a style break imo compared the other classes which follow Java-bean style. Is that something we should change for the release? I'm definitely fine with keeping it the way it is, but it's something I've noticed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69278183
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    --- End diff --
    
    It's also quite big and blows up the Basic API page. It takes up a quarter of the page alone.
    
    I'll move it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69173750
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    +      @Override
    +      public Integer getValue() {
    +        return valueToExpose;
    +      }});
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Histogram
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.histogram = getRuntimeContext().getMetricGroup().histogram("myHistogram", new MyHistogram());
    +    ...
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +    ...
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +This wrapper is contained in the `flink-metrics-dropwizard` module.
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represents the entities it is tied to. By default a metric that is registered in a user function will be scoped to the operator in which the function runs, the task/job it belongs to and the taskManager/host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter()` call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +  - Applied to all metrics that were scoped to a jobmanager.
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +  - Applied to all metrics that were scoped to a jobmanager and job.
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +  - Applied to all metrics that were scoped to a taskmanager.
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +  - Applied to all metrics that were scoped to a taskmanager and job.
    +- `metrics.scope.tm.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +  - Applied to all metrics that were scoped to an operator.
    +
    +Note that for metrics for which multiple formats may apply (like jm and jm.job) the most specific format takes precedence,
    +in this case jm.job.
    +
    +The hierarchical orders are as follows:
    +
    +jm < jm.job 
    +
    +tm < tm.job < tm.task < tm.operator
    +
    +This hierarchy also defines which variables may be accessed. The `tm.operator` format may contain variables for jobs, whereas `tm.job` may not contain
    +variables for operators. There is no restriction on order of variables.
    +
    +There is no restriction on the order of variables.
    +
    +The default scope for operator metrics will result in a metric name akin to `localhost.taskmanager.1234.MyJob.MyOperator.0`
    +
    +If you also want to include the task name, but omit the taskmanager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: \<host>.\<job_name>.\<task_name>.\<operator_name>.\<subtask_index>`
    +
    +This could create the name `localhost.MyJob.MySource_->_MyOperator.MyOperator.0`.
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +### Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in the `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS
    +
    +You can write your own `Reporter` by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the `Scheduled` interface as well.
    +
    +By default Flink uses JMX to expose metrics.
    +All non-JMXReporters are not part of the distribution and have to be added to the classpath manually, either by putting the jar into /lib
    +or including it in the job jar.
    +
    +Flink supports the following systems:
    +
    +#### JMX
    +
    +The port for JMX can be configured by setting the `metrics.jmx.port` key. This parameter expects either a single port
    +or a port range, with the default being 9010-9025. The used port is shown in the relevant Job-/TaskManager log.
    +
    +#### Ganglia (org.apache.flink.metrics.ganglia.GangliaReporter)
    +Dependency:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-ganglia</artifactId>
    +      <version>1.1-SNAPSHOT</version>
    +</dependency>
    +{% endhighlight %}
    +
    +Parameters:
    --- End diff --
    
    yes


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by uce <gi...@git.apache.org>.
Github user uce commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69163863
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    +      @Override
    +      public Integer getValue() {
    +        return valueToExpose;
    +      }});
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Histogram
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.histogram = getRuntimeContext().getMetricGroup().histogram("myHistogram", new MyHistogram());
    +    ...
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +    ...
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +This wrapper is contained in the `flink-metrics-dropwizard` module.
    --- End diff --
    
    - Link to how a user can add this module
    - I would also add an example for this actually (as everyone who wants to use Histogram will need this)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68563676
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter() call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +- `metrics.scope.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +- `metrics.scope.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +There is no restriction on the order of variables.
    +
    +Which format is applied to a metric depends on which entities it was scoped to.
    +The `metrics.scope.operator` format will be applied to all metrics that were scoped to an Operator.
    +The `metrics.scope.tm.job` format will be applied to all metrics that were only scoped to a Job and a TaskManager.
    --- End diff --
    
    job and task manager.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69910710
  
    --- Diff: docs/apis/metrics.md ---
    @@ -0,0 +1,441 @@
    +---
    +title: "Metrics"
    +# Top-level navigation
    +top-nav-group: apis
    +top-nav-pos: 13
    +top-nav-title: "Metrics"
    +---
    +<!--
    +Licensed to the Apache Software Foundation (ASF) under one
    +or more contributor license agreements.  See the NOTICE file
    +distributed with this work for additional information
    +regarding copyright ownership.  The ASF licenses this file
    +to you under the Apache License, Version 2.0 (the
    +"License"); you may not use this file except in compliance
    +with the License.  You may obtain a copy of the License at
    +
    +  http://www.apache.org/licenses/LICENSE-2.0
    +
    +Unless required by applicable law or agreed to in writing,
    +software distributed under the License is distributed on an
    +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    +KIND, either express or implied.  See the License for the
    +specific language governing permissions and limitations
    +under the License.
    +-->
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +* This will be replaced by the TOC
    +{:toc}
    +
    +## Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.counter = getRuntimeContext()
    +      .getMetricGroup()
    +      .counter("myCounter");
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    this.counter.inc();
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.counter = getRuntimeContext()
    +      .getMetricGroup()
    +      .counter("myCustomCounter", new CustomCounter());
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is no restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    getRuntimeContext()
    +      .getMetricGroup()
    +      .gauge("MyGauge", new Gauge<Integer>() {
    +        @Override
    +        public Integer getValue() {
    +          return valueToExpose;
    +        }
    +      });
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Note that reporters will turn the exposed object into a `String`, which means that a meaningful `toString()` implementation is required.
    +
    +#### Histogram
    +
    +A `Histogram` measures the distribution of long values.
    +You can register one by calling `histogram(String name, Histogram histogram)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.histogram = getRuntimeContext()
    +      .getMetricGroup()
    +      .histogram("myHistogram", new MyHistogram());
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink does not provide a default implementation for `Histogram`, but offers a {% gh_link flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/DropwizardHistogramWrapper.java "Wrapper" %} that allows usage of Codahale/DropWizard histograms.
    +To use this wrapper add the following dependency in your `pom.xml`:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-dropwizard</artifactId>
    +      <version>{{site.version}}</version>
    +</dependency>
    +{% endhighlight %}
    +
    +You can then register a Codahale/DropWizard histogram like this:
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    com.codahale.metrics.Histogram histogram =
    +      new com.codahale.metrics.Histogram(new SlidingWindowReservoir(500));
    +
    +    this.histogram = getRuntimeContext()
    +      .getMetricGroup()
    +      .histogram("myHistogram", new DropWizardHistogramWrapper(histogram));
    +  }
    +}
    +{% endhighlight %}
    +
    +## Scope
    +
    +Every metric is assigned an identifier under which it will be reported that is based on 3 components: the user-provided name when registering the metric, an optional user-defined scope and a system-provided scope.
    +For example, if `A.B` is the sytem scope, `C.D` the user scope and `E` the name, then the identifier for the metric will be `A.B.C.D.E`.
    +
    +### User Scope
    +
    +You can define a user scope by calling either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext()
    +  .getMetricGroup()
    +  .addGroup("MyMetrics")
    +  .counter("myCounter");
    +
    +{% endhighlight %}
    +
    +### System Scope
    +
    +The system scope contains context information about the metric, for example in which task it was registered or what job that task belongs to.
    +
    +Which context information should be included can be configured by setting the following keys in `conf/flink-conf.yaml`.
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "&lt;task_id&gt;") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: &lt;host&gt;.jobmanager
    +  - Applied to all metrics that were scoped to a job manager.
    +- `metrics.scope.jm.job`
    +  - Default: &lt;host&gt;.jobmanager.&lt;job_name&gt;
    +  - Applied to all metrics that were scoped to a job manager and job.
    +- `metrics.scope.tm`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;
    +  - Applied to all metrics that were scoped to a task manager.
    +- `metrics.scope.tm.job`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;
    +  - Applied to all metrics that were scoped to a task manager and job.
    +- `metrics.scope.tm.task`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;.&lt;task_name&gt;.&lt;subtask_index&gt;
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;.&lt;operator_name&gt;.&lt;subtask_index&gt;
    +  - Applied to all metrics that were scoped to an operator.
    +
    +There are no restrictions on the number or order of variables. Variables are case sensitive.
    +
    +The default scope for operator metrics will result in an identifier akin to `localhost.taskmanager.1234.MyJob.MyOperator.0.MyMetric`
    +
    +If you also want to include the task name but omit the task manager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: <host>.<job_name>.<task_name>.<operator_name>.<subtask_index>`
    +
    +This could create the identifier `localhost.MyJob.MySource_->_MyOperator.MyOperator.0.MyMetric`.
    +
    +Note that for this format string an identifier clash can occur should the same job be run multiple times concurrently, which can lead to inconsistent metric data.
    +As such it is advised to either use format strings that provide a certain degree of uniqueness by including IDs (e.g &lt;job_id&gt;)
    +or by assigning unique names to jobs and operators.
    +
    +### List of all Variables
    +
    +- JobManager: &lt;host&gt;
    +- TaskManager: &lt;host&gt;, &lt;tm_id&gt;
    +- Job: &lt;job_id&gt;, &lt;job_name&gt;
    +- Task: &lt;task_id&gt;, &lt;task_name&gt;, &lt;task_attempt_id&gt;, &lt;task_attempt_num&gt;, &lt;subtask_index&gt;
    +- Operator: &lt;operator_name&gt;, &lt;subtask_index&gt;
    +
    +## Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS
    +
    +You can write your own `Reporter` by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the `Scheduled` interface as well.
    +
    +By default Flink uses JMX to expose metrics.
    +All non-JMXReporters are not part of the distribution and have to be added [manually]({{site.baseurl}}/apis/cluster_execution.html#linking-with-modules-not-contained-in-the-binary-distribution).
    +
    +The following sections list the supported reporters.
    +
    +### JMX
    +
    +The port for JMX can be configured by setting the `metrics.jmx.port` key. This parameter expects either a single port
    +or a port range, with the default being 9010-9025. The used port is shown in the relevant job or task manager log.
    +
    +### Ganglia (org.apache.flink.metrics.ganglia.GangliaReporter)
    +Dependency:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-ganglia</artifactId>
    +      <version>{{site.version}}</version>
    +</dependency>
    --- End diff --
    
    can we fix this somehow? I would prefer it if users were not **required** to juggle around jars.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by uce <gi...@git.apache.org>.
Github user uce commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69165872
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    +      @Override
    +      public Integer getValue() {
    +        return valueToExpose;
    +      }});
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Histogram
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.histogram = getRuntimeContext().getMetricGroup().histogram("myHistogram", new MyHistogram());
    +    ...
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +    ...
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +This wrapper is contained in the `flink-metrics-dropwizard` module.
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represents the entities it is tied to. By default a metric that is registered in a user function will be scoped to the operator in which the function runs, the task/job it belongs to and the taskManager/host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter()` call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +  - Applied to all metrics that were scoped to a jobmanager.
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +  - Applied to all metrics that were scoped to a jobmanager and job.
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +  - Applied to all metrics that were scoped to a taskmanager.
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +  - Applied to all metrics that were scoped to a taskmanager and job.
    +- `metrics.scope.tm.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +  - Applied to all metrics that were scoped to an operator.
    +
    +Note that for metrics for which multiple formats may apply (like jm and jm.job) the most specific format takes precedence,
    +in this case jm.job.
    +
    +The hierarchical orders are as follows:
    +
    +jm < jm.job 
    +
    +tm < tm.job < tm.task < tm.operator
    +
    +This hierarchy also defines which variables may be accessed. The `tm.operator` format may contain variables for jobs, whereas `tm.job` may not contain
    +variables for operators. There is no restriction on order of variables.
    +
    +There is no restriction on the order of variables.
    +
    +The default scope for operator metrics will result in a metric name akin to `localhost.taskmanager.1234.MyJob.MyOperator.0`
    +
    +If you also want to include the task name, but omit the taskmanager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: \<host>.\<job_name>.\<task_name>.\<operator_name>.\<subtask_index>`
    +
    +This could create the name `localhost.MyJob.MySource_->_MyOperator.MyOperator.0`.
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +### Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in the `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS
    +
    +You can write your own `Reporter` by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the `Scheduled` interface as well.
    --- End diff --
    
    Are there any reporters which are not scheduled?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68563419
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter() call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    --- End diff --
    
    The config key definition should also be mirrored to https://ci.apache.org/projects/flink/flink-docs-master/setup/config.html. Actually I think that the config keys should be moved from `MetricRegistry` to `ConfigConstants`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r70246597
  
    --- Diff: docs/apis/metrics.md ---
    @@ -0,0 +1,441 @@
    +---
    +title: "Metrics"
    +# Top-level navigation
    +top-nav-group: apis
    +top-nav-pos: 13
    +top-nav-title: "Metrics"
    +---
    +<!--
    +Licensed to the Apache Software Foundation (ASF) under one
    +or more contributor license agreements.  See the NOTICE file
    +distributed with this work for additional information
    +regarding copyright ownership.  The ASF licenses this file
    +to you under the Apache License, Version 2.0 (the
    +"License"); you may not use this file except in compliance
    +with the License.  You may obtain a copy of the License at
    +
    +  http://www.apache.org/licenses/LICENSE-2.0
    +
    +Unless required by applicable law or agreed to in writing,
    +software distributed under the License is distributed on an
    +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    +KIND, either express or implied.  See the License for the
    +specific language governing permissions and limitations
    +under the License.
    +-->
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +* This will be replaced by the TOC
    +{:toc}
    +
    +## Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.counter = getRuntimeContext()
    +      .getMetricGroup()
    +      .counter("myCounter");
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    this.counter.inc();
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.counter = getRuntimeContext()
    +      .getMetricGroup()
    +      .counter("myCustomCounter", new CustomCounter());
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is no restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    getRuntimeContext()
    +      .getMetricGroup()
    +      .gauge("MyGauge", new Gauge<Integer>() {
    +        @Override
    +        public Integer getValue() {
    +          return valueToExpose;
    +        }
    +      });
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Note that reporters will turn the exposed object into a `String`, which means that a meaningful `toString()` implementation is required.
    +
    +#### Histogram
    +
    +A `Histogram` measures the distribution of long values.
    +You can register one by calling `histogram(String name, Histogram histogram)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.histogram = getRuntimeContext()
    +      .getMetricGroup()
    +      .histogram("myHistogram", new MyHistogram());
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink does not provide a default implementation for `Histogram`, but offers a {% gh_link flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/DropwizardHistogramWrapper.java "Wrapper" %} that allows usage of Codahale/DropWizard histograms.
    +To use this wrapper add the following dependency in your `pom.xml`:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-dropwizard</artifactId>
    +      <version>{{site.version}}</version>
    +</dependency>
    +{% endhighlight %}
    +
    +You can then register a Codahale/DropWizard histogram like this:
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    com.codahale.metrics.Histogram histogram =
    +      new com.codahale.metrics.Histogram(new SlidingWindowReservoir(500));
    +
    +    this.histogram = getRuntimeContext()
    +      .getMetricGroup()
    +      .histogram("myHistogram", new DropWizardHistogramWrapper(histogram));
    +  }
    +}
    +{% endhighlight %}
    +
    +## Scope
    +
    +Every metric is assigned an identifier under which it will be reported that is based on 3 components: the user-provided name when registering the metric, an optional user-defined scope and a system-provided scope.
    +For example, if `A.B` is the sytem scope, `C.D` the user scope and `E` the name, then the identifier for the metric will be `A.B.C.D.E`.
    +
    +### User Scope
    +
    +You can define a user scope by calling either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext()
    +  .getMetricGroup()
    +  .addGroup("MyMetrics")
    +  .counter("myCounter");
    +
    +{% endhighlight %}
    +
    +### System Scope
    +
    +The system scope contains context information about the metric, for example in which task it was registered or what job that task belongs to.
    +
    +Which context information should be included can be configured by setting the following keys in `conf/flink-conf.yaml`.
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "&lt;task_id&gt;") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: &lt;host&gt;.jobmanager
    +  - Applied to all metrics that were scoped to a job manager.
    +- `metrics.scope.jm.job`
    +  - Default: &lt;host&gt;.jobmanager.&lt;job_name&gt;
    +  - Applied to all metrics that were scoped to a job manager and job.
    +- `metrics.scope.tm`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;
    +  - Applied to all metrics that were scoped to a task manager.
    +- `metrics.scope.tm.job`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;
    +  - Applied to all metrics that were scoped to a task manager and job.
    +- `metrics.scope.tm.task`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;.&lt;task_name&gt;.&lt;subtask_index&gt;
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;.&lt;operator_name&gt;.&lt;subtask_index&gt;
    +  - Applied to all metrics that were scoped to an operator.
    +
    +There are no restrictions on the number or order of variables. Variables are case sensitive.
    +
    +The default scope for operator metrics will result in an identifier akin to `localhost.taskmanager.1234.MyJob.MyOperator.0.MyMetric`
    +
    +If you also want to include the task name but omit the task manager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: <host>.<job_name>.<task_name>.<operator_name>.<subtask_index>`
    +
    +This could create the identifier `localhost.MyJob.MySource_->_MyOperator.MyOperator.0.MyMetric`.
    +
    +Note that for this format string an identifier clash can occur should the same job be run multiple times concurrently, which can lead to inconsistent metric data.
    +As such it is advised to either use format strings that provide a certain degree of uniqueness by including IDs (e.g &lt;job_id&gt;)
    +or by assigning unique names to jobs and operators.
    +
    +### List of all Variables
    +
    +- JobManager: &lt;host&gt;
    +- TaskManager: &lt;host&gt;, &lt;tm_id&gt;
    +- Job: &lt;job_id&gt;, &lt;job_name&gt;
    +- Task: &lt;task_id&gt;, &lt;task_name&gt;, &lt;task_attempt_id&gt;, &lt;task_attempt_num&gt;, &lt;subtask_index&gt;
    +- Operator: &lt;operator_name&gt;, &lt;subtask_index&gt;
    +
    +## Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS
    +
    +You can write your own `Reporter` by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the `Scheduled` interface as well.
    +
    +By default Flink uses JMX to expose metrics.
    +All non-JMXReporters are not part of the distribution. To use them you have to copy the respective fat jar to the `/lib` folder.
    +
    +The following sections list the supported reporters.
    +
    +### JMX
    +
    +The port for JMX can be configured by setting the `metrics.jmx.port` key. This parameter expects either a single port
    +or a port range, with the default being 9010-9025. The used port is shown in the relevant job or task manager log.
    +
    +### Ganglia (org.apache.flink.metrics.ganglia.GangliaReporter)
    +Dependency:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-ganglia</artifactId>
    +      <version>{{site.version}}</version>
    +</dependency>
    +{% endhighlight %}
    +
    +Parameters:
    +
    +- `host` - the ganglia server host
    --- End diff --
    
    The `udp_recv_channel.bind` is set per default in the `gmond.conf` if you set up Ganglia.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68563243
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter() call. The order is always \<system_scope>\<user_scope>\<name>.
    --- End diff --
    
    Don't we insert a scope separating character in between the system, user and name scope?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68562801
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    --- End diff --
    
    I think it would be better to present the different metric types either as a descriptive list similar to the description environment of latex or by using sub headings. It could also be more descriptive to give for each metric type a short code snippet how to register and interacting with it. That would explain their interface which is important for the user.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68904915
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter() call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    --- End diff --
    
    I agree that we should mirror the config keys to the configuration page as well + move the config keys.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68561771
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    --- End diff --
    
    "allows users to gather and ..." or "allows user gathering and ..."


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r70123498
  
    --- Diff: docs/apis/metrics.md ---
    @@ -0,0 +1,441 @@
    +---
    +title: "Metrics"
    +# Top-level navigation
    +top-nav-group: apis
    +top-nav-pos: 13
    +top-nav-title: "Metrics"
    +---
    +<!--
    +Licensed to the Apache Software Foundation (ASF) under one
    +or more contributor license agreements.  See the NOTICE file
    +distributed with this work for additional information
    +regarding copyright ownership.  The ASF licenses this file
    +to you under the Apache License, Version 2.0 (the
    +"License"); you may not use this file except in compliance
    +with the License.  You may obtain a copy of the License at
    +
    +  http://www.apache.org/licenses/LICENSE-2.0
    +
    +Unless required by applicable law or agreed to in writing,
    +software distributed under the License is distributed on an
    +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    +KIND, either express or implied.  See the License for the
    +specific language governing permissions and limitations
    +under the License.
    +-->
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +* This will be replaced by the TOC
    +{:toc}
    +
    +## Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.counter = getRuntimeContext()
    +      .getMetricGroup()
    +      .counter("myCounter");
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    this.counter.inc();
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.counter = getRuntimeContext()
    +      .getMetricGroup()
    +      .counter("myCustomCounter", new CustomCounter());
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is no restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    getRuntimeContext()
    +      .getMetricGroup()
    +      .gauge("MyGauge", new Gauge<Integer>() {
    +        @Override
    +        public Integer getValue() {
    +          return valueToExpose;
    +        }
    +      });
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Note that reporters will turn the exposed object into a `String`, which means that a meaningful `toString()` implementation is required.
    +
    +#### Histogram
    +
    +A `Histogram` measures the distribution of long values.
    +You can register one by calling `histogram(String name, Histogram histogram)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.histogram = getRuntimeContext()
    +      .getMetricGroup()
    +      .histogram("myHistogram", new MyHistogram());
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink does not provide a default implementation for `Histogram`, but offers a {% gh_link flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/DropwizardHistogramWrapper.java "Wrapper" %} that allows usage of Codahale/DropWizard histograms.
    +To use this wrapper add the following dependency in your `pom.xml`:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-dropwizard</artifactId>
    +      <version>{{site.version}}</version>
    +</dependency>
    +{% endhighlight %}
    +
    +You can then register a Codahale/DropWizard histogram like this:
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    com.codahale.metrics.Histogram histogram =
    +      new com.codahale.metrics.Histogram(new SlidingWindowReservoir(500));
    +
    +    this.histogram = getRuntimeContext()
    +      .getMetricGroup()
    +      .histogram("myHistogram", new DropWizardHistogramWrapper(histogram));
    +  }
    +}
    +{% endhighlight %}
    +
    +## Scope
    +
    +Every metric is assigned an identifier under which it will be reported that is based on 3 components: the user-provided name when registering the metric, an optional user-defined scope and a system-provided scope.
    +For example, if `A.B` is the sytem scope, `C.D` the user scope and `E` the name, then the identifier for the metric will be `A.B.C.D.E`.
    +
    +### User Scope
    +
    +You can define a user scope by calling either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext()
    +  .getMetricGroup()
    +  .addGroup("MyMetrics")
    +  .counter("myCounter");
    +
    +{% endhighlight %}
    +
    +### System Scope
    +
    +The system scope contains context information about the metric, for example in which task it was registered or what job that task belongs to.
    +
    +Which context information should be included can be configured by setting the following keys in `conf/flink-conf.yaml`.
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "&lt;task_id&gt;") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: &lt;host&gt;.jobmanager
    +  - Applied to all metrics that were scoped to a job manager.
    +- `metrics.scope.jm.job`
    +  - Default: &lt;host&gt;.jobmanager.&lt;job_name&gt;
    +  - Applied to all metrics that were scoped to a job manager and job.
    +- `metrics.scope.tm`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;
    +  - Applied to all metrics that were scoped to a task manager.
    +- `metrics.scope.tm.job`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;
    +  - Applied to all metrics that were scoped to a task manager and job.
    +- `metrics.scope.tm.task`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;.&lt;task_name&gt;.&lt;subtask_index&gt;
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;.&lt;operator_name&gt;.&lt;subtask_index&gt;
    +  - Applied to all metrics that were scoped to an operator.
    +
    +There are no restrictions on the number or order of variables. Variables are case sensitive.
    +
    +The default scope for operator metrics will result in an identifier akin to `localhost.taskmanager.1234.MyJob.MyOperator.0.MyMetric`
    +
    +If you also want to include the task name but omit the task manager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: <host>.<job_name>.<task_name>.<operator_name>.<subtask_index>`
    +
    +This could create the identifier `localhost.MyJob.MySource_->_MyOperator.MyOperator.0.MyMetric`.
    +
    +Note that for this format string an identifier clash can occur should the same job be run multiple times concurrently, which can lead to inconsistent metric data.
    +As such it is advised to either use format strings that provide a certain degree of uniqueness by including IDs (e.g &lt;job_id&gt;)
    +or by assigning unique names to jobs and operators.
    +
    +### List of all Variables
    +
    +- JobManager: &lt;host&gt;
    +- TaskManager: &lt;host&gt;, &lt;tm_id&gt;
    +- Job: &lt;job_id&gt;, &lt;job_name&gt;
    +- Task: &lt;task_id&gt;, &lt;task_name&gt;, &lt;task_attempt_id&gt;, &lt;task_attempt_num&gt;, &lt;subtask_index&gt;
    +- Operator: &lt;operator_name&gt;, &lt;subtask_index&gt;
    +
    +## Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS
    +
    +You can write your own `Reporter` by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the `Scheduled` interface as well.
    +
    +By default Flink uses JMX to expose metrics.
    +All non-JMXReporters are not part of the distribution. To use them you have to copy the respective fat jar to the `/lib` folder.
    +
    +The following sections list the supported reporters.
    +
    +### JMX
    +
    +The port for JMX can be configured by setting the `metrics.jmx.port` key. This parameter expects either a single port
    +or a port range, with the default being 9010-9025. The used port is shown in the relevant job or task manager log.
    +
    +### Ganglia (org.apache.flink.metrics.ganglia.GangliaReporter)
    +Dependency:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-ganglia</artifactId>
    +      <version>{{site.version}}</version>
    +</dependency>
    +{% endhighlight %}
    +
    +Parameters:
    +
    +- `host` - the ganglia server host
    --- End diff --
    
    Note that it is not mandatory to set the bind property. If it is not set then generally host should be set to localhost.
    
    But yes, if it set it to something but localhost then naturally you have to set it to that value, will include that in the docs.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69913171
  
    --- Diff: docs/apis/metrics.md ---
    @@ -0,0 +1,441 @@
    +---
    +title: "Metrics"
    +# Top-level navigation
    +top-nav-group: apis
    +top-nav-pos: 13
    +top-nav-title: "Metrics"
    +---
    +<!--
    +Licensed to the Apache Software Foundation (ASF) under one
    +or more contributor license agreements.  See the NOTICE file
    +distributed with this work for additional information
    +regarding copyright ownership.  The ASF licenses this file
    +to you under the Apache License, Version 2.0 (the
    +"License"); you may not use this file except in compliance
    +with the License.  You may obtain a copy of the License at
    +
    +  http://www.apache.org/licenses/LICENSE-2.0
    +
    +Unless required by applicable law or agreed to in writing,
    +software distributed under the License is distributed on an
    +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    +KIND, either express or implied.  See the License for the
    +specific language governing permissions and limitations
    +under the License.
    +-->
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +* This will be replaced by the TOC
    +{:toc}
    +
    +## Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.counter = getRuntimeContext()
    +      .getMetricGroup()
    +      .counter("myCounter");
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    this.counter.inc();
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.counter = getRuntimeContext()
    +      .getMetricGroup()
    +      .counter("myCustomCounter", new CustomCounter());
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is no restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    getRuntimeContext()
    +      .getMetricGroup()
    +      .gauge("MyGauge", new Gauge<Integer>() {
    +        @Override
    +        public Integer getValue() {
    +          return valueToExpose;
    +        }
    +      });
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Note that reporters will turn the exposed object into a `String`, which means that a meaningful `toString()` implementation is required.
    +
    +#### Histogram
    +
    +A `Histogram` measures the distribution of long values.
    +You can register one by calling `histogram(String name, Histogram histogram)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.histogram = getRuntimeContext()
    +      .getMetricGroup()
    +      .histogram("myHistogram", new MyHistogram());
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink does not provide a default implementation for `Histogram`, but offers a {% gh_link flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/DropwizardHistogramWrapper.java "Wrapper" %} that allows usage of Codahale/DropWizard histograms.
    +To use this wrapper add the following dependency in your `pom.xml`:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-dropwizard</artifactId>
    +      <version>{{site.version}}</version>
    +</dependency>
    +{% endhighlight %}
    +
    +You can then register a Codahale/DropWizard histogram like this:
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    com.codahale.metrics.Histogram histogram =
    +      new com.codahale.metrics.Histogram(new SlidingWindowReservoir(500));
    +
    +    this.histogram = getRuntimeContext()
    +      .getMetricGroup()
    +      .histogram("myHistogram", new DropWizardHistogramWrapper(histogram));
    +  }
    +}
    +{% endhighlight %}
    +
    +## Scope
    +
    +Every metric is assigned an identifier under which it will be reported that is based on 3 components: the user-provided name when registering the metric, an optional user-defined scope and a system-provided scope.
    +For example, if `A.B` is the sytem scope, `C.D` the user scope and `E` the name, then the identifier for the metric will be `A.B.C.D.E`.
    +
    +### User Scope
    +
    +You can define a user scope by calling either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext()
    +  .getMetricGroup()
    +  .addGroup("MyMetrics")
    +  .counter("myCounter");
    +
    +{% endhighlight %}
    +
    +### System Scope
    +
    +The system scope contains context information about the metric, for example in which task it was registered or what job that task belongs to.
    +
    +Which context information should be included can be configured by setting the following keys in `conf/flink-conf.yaml`.
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "&lt;task_id&gt;") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: &lt;host&gt;.jobmanager
    +  - Applied to all metrics that were scoped to a job manager.
    +- `metrics.scope.jm.job`
    +  - Default: &lt;host&gt;.jobmanager.&lt;job_name&gt;
    +  - Applied to all metrics that were scoped to a job manager and job.
    +- `metrics.scope.tm`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;
    +  - Applied to all metrics that were scoped to a task manager.
    +- `metrics.scope.tm.job`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;
    +  - Applied to all metrics that were scoped to a task manager and job.
    +- `metrics.scope.tm.task`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;.&lt;task_name&gt;.&lt;subtask_index&gt;
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;.&lt;operator_name&gt;.&lt;subtask_index&gt;
    +  - Applied to all metrics that were scoped to an operator.
    +
    +There are no restrictions on the number or order of variables. Variables are case sensitive.
    +
    +The default scope for operator metrics will result in an identifier akin to `localhost.taskmanager.1234.MyJob.MyOperator.0.MyMetric`
    +
    +If you also want to include the task name but omit the task manager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: <host>.<job_name>.<task_name>.<operator_name>.<subtask_index>`
    +
    +This could create the identifier `localhost.MyJob.MySource_->_MyOperator.MyOperator.0.MyMetric`.
    +
    +Note that for this format string an identifier clash can occur should the same job be run multiple times concurrently, which can lead to inconsistent metric data.
    +As such it is advised to either use format strings that provide a certain degree of uniqueness by including IDs (e.g &lt;job_id&gt;)
    +or by assigning unique names to jobs and operators.
    +
    +### List of all Variables
    +
    +- JobManager: &lt;host&gt;
    +- TaskManager: &lt;host&gt;, &lt;tm_id&gt;
    +- Job: &lt;job_id&gt;, &lt;job_name&gt;
    +- Task: &lt;task_id&gt;, &lt;task_name&gt;, &lt;task_attempt_id&gt;, &lt;task_attempt_num&gt;, &lt;subtask_index&gt;
    +- Operator: &lt;operator_name&gt;, &lt;subtask_index&gt;
    +
    +## Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS
    +
    +You can write your own `Reporter` by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the `Scheduled` interface as well.
    +
    +By default Flink uses JMX to expose metrics.
    +All non-JMXReporters are not part of the distribution and have to be added [manually]({{site.baseurl}}/apis/cluster_execution.html#linking-with-modules-not-contained-in-the-binary-distribution).
    +
    +The following sections list the supported reporters.
    +
    +### JMX
    +
    +The port for JMX can be configured by setting the `metrics.jmx.port` key. This parameter expects either a single port
    +or a port range, with the default being 9010-9025. The used port is shown in the relevant job or task manager log.
    +
    +### Ganglia (org.apache.flink.metrics.ganglia.GangliaReporter)
    +Dependency:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-ganglia</artifactId>
    +      <version>{{site.version}}</version>
    +</dependency>
    --- End diff --
    
    Note: we probably need this long-term anyway (for per-job reporters)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68562093
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    --- End diff --
    
    Maybe we should also demonstrate how to register a custom metric.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by uce <gi...@git.apache.org>.
Github user uce commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69161650
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    --- End diff --
    
    I would make this a separate page under `Programming Guide`. It's guide hidden under the `Basic API` concepts although I understand that it it is technically the right place for it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68563067
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    --- End diff --
    
    Same for task manager/host


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68563905
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter() call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +- `metrics.scope.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +- `metrics.scope.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +There is no restriction on the order of variables.
    +
    +Which format is applied to a metric depends on which entities it was scoped to.
    +The `metrics.scope.operator` format will be applied to all metrics that were scoped to an Operator.
    +The `metrics.scope.tm.job` format will be applied to all metrics that were only scoped to a Job and a TaskManager.
    +
    +This allows you to define explicitly how metrics on every level should be reported.
    +
    +### Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in the `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter)
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010)
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS)
    +
    +By default Flink uses JMX to expose metrics.
    --- End diff --
    
    JMX should also get a sub heading.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68567416
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter() call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +- `metrics.scope.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +- `metrics.scope.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +There is no restriction on the order of variables.
    +
    +Which format is applied to a metric depends on which entities it was scoped to.
    +The `metrics.scope.operator` format will be applied to all metrics that were scoped to an Operator.
    +The `metrics.scope.tm.job` format will be applied to all metrics that were only scoped to a Job and a TaskManager.
    +
    +This allows you to define explicitly how metrics on every level should be reported.
    +
    +### Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in the `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter)
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010)
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS)
    +
    +By default Flink uses JMX to expose metrics.
    +
    +The port for JMX can be configured by setting the `metrics.jmx.port` key. This parameter expects either a single port
    +or a port range, with the default being 9010-9025. Which port is used in the end is shown in the relevant Job-/TaskManager log.
    +
    +All non-JMXReporters are not part of the distribution and have to be added to the classpath manually. (usually by putting the jar into /lib)
    +
    +You can write your own Reporter by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the Scheduled interface as well.
    --- End diff --
    
    Doesn't 'If the Reporter should send out reports regularly...' indicate what `Scheduled` means?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68562330
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    --- End diff --
    
    register without *s*


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by uce <gi...@git.apache.org>.
Github user uce commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69166536
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    +      @Override
    +      public Integer getValue() {
    +        return valueToExpose;
    +      }});
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Histogram
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.histogram = getRuntimeContext().getMetricGroup().histogram("myHistogram", new MyHistogram());
    +    ...
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +    ...
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +This wrapper is contained in the `flink-metrics-dropwizard` module.
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represents the entities it is tied to. By default a metric that is registered in a user function will be scoped to the operator in which the function runs, the task/job it belongs to and the taskManager/host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter()` call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +  - Applied to all metrics that were scoped to a jobmanager.
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +  - Applied to all metrics that were scoped to a jobmanager and job.
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +  - Applied to all metrics that were scoped to a taskmanager.
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +  - Applied to all metrics that were scoped to a taskmanager and job.
    +- `metrics.scope.tm.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +  - Applied to all metrics that were scoped to an operator.
    +
    +Note that for metrics for which multiple formats may apply (like jm and jm.job) the most specific format takes precedence,
    +in this case jm.job.
    +
    +The hierarchical orders are as follows:
    +
    +jm < jm.job 
    +
    +tm < tm.job < tm.task < tm.operator
    +
    +This hierarchy also defines which variables may be accessed. The `tm.operator` format may contain variables for jobs, whereas `tm.job` may not contain
    +variables for operators. There is no restriction on order of variables.
    +
    +There is no restriction on the order of variables.
    +
    +The default scope for operator metrics will result in a metric name akin to `localhost.taskmanager.1234.MyJob.MyOperator.0`
    +
    +If you also want to include the task name, but omit the taskmanager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: \<host>.\<job_name>.\<task_name>.\<operator_name>.\<subtask_index>`
    +
    +This could create the name `localhost.MyJob.MySource_->_MyOperator.MyOperator.0`.
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +### Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in the `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS
    +
    +You can write your own `Reporter` by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the `Scheduled` interface as well.
    +
    +By default Flink uses JMX to expose metrics.
    +All non-JMXReporters are not part of the distribution and have to be added to the classpath manually, either by putting the jar into /lib
    +or including it in the job jar.
    +
    +Flink supports the following systems:
    +
    +#### JMX
    +
    +The port for JMX can be configured by setting the `metrics.jmx.port` key. This parameter expects either a single port
    +or a port range, with the default being 9010-9025. The used port is shown in the relevant Job-/TaskManager log.
    +
    +#### Ganglia (org.apache.flink.metrics.ganglia.GangliaReporter)
    +Dependency:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-ganglia</artifactId>
    +      <version>1.1-SNAPSHOT</version>
    +</dependency>
    +{% endhighlight %}
    +
    +Parameters:
    --- End diff --
    
    Do I need to specify these via `metrics.reporter.arguments`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68564604
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter() call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +- `metrics.scope.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +- `metrics.scope.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +There is no restriction on the order of variables.
    +
    +Which format is applied to a metric depends on which entities it was scoped to.
    +The `metrics.scope.operator` format will be applied to all metrics that were scoped to an Operator.
    +The `metrics.scope.tm.job` format will be applied to all metrics that were only scoped to a Job and a TaskManager.
    +
    +This allows you to define explicitly how metrics on every level should be reported.
    +
    +### Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in the `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter)
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010)
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS)
    +
    +By default Flink uses JMX to expose metrics.
    +
    +The port for JMX can be configured by setting the `metrics.jmx.port` key. This parameter expects either a single port
    +or a port range, with the default being 9010-9025. Which port is used in the end is shown in the relevant Job-/TaskManager log.
    +
    +All non-JMXReporters are not part of the distribution and have to be added to the classpath manually. (usually by putting the jar into /lib)
    +
    +You can write your own Reporter by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the Scheduled interface as well.
    +
    +Flink supports the following systems with their respective configuration parameters:
    +
    +#### Ganglia (org.apcahe.flink.metrics.ganglia.GangliaReporter)
    +- `host`
    +- `port`
    +- `dmax`
    +- `tmax`
    +- `ttl`
    +- `addressingMode` (UNICAST/MULTICAST)
    +
    +#### Graphite (org.apache.flink.metrics.graphite.GraphiteReporter)
    +- `host`
    +- `port`
    +
    +#### StatsD (org.apache.flink.metrics.statsd.StatsDReporter)
    +- `host`
    +- `port`
    +
    --- End diff --
    
    We should also say how the different reporters expose certain metrics. E.g. the histogram metric reports the mean, median, percentiles etc. The names under which this happens might differ between different reporters.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by uce <gi...@git.apache.org>.
Github user uce commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69163034
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    +      @Override
    +      public Integer getValue() {
    +        return valueToExpose;
    +      }});
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Histogram
    +
    +A Histogram measure the distribution of long values.
    --- End diff --
    
    measureS with s


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68563125
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    --- End diff --
    
    explicit enumeration of the different `addGroup` methods is better imo.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68981838
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    +      @Override
    +      public Integer getValue() {
    +        return valueToExpose;
    +      }});
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Histogram
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.histogram = getRuntimeContext().getMetricGroup().histogram("myHistogram", new MyHistogram());
    +    ...
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +    ...
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +This wrapper is contained in the `flink-metrics-dropwizard` module.
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represents the entities it is tied to. By default a metric that is registered in a user function will be scoped to the operator in which the function runs, the task/job it belongs to and the taskManager/host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter()` call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +  - Applied to all metrics that were scoped to a jobmanager.
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +  - Applied to all metrics that were scoped to a jobmanager and job.
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +  - Applied to all metrics that were scoped to a taskmanager.
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +  - Applied to all metrics that were scoped to a taskmanager and job.
    +- `metrics.scope.tm.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +  - Applied to all metrics that were scoped to an operator.
    +
    +Note that for metrics for which multiple formats may apply (like jm and jm.job) the most specific format takes precedence,
    +in this case jm.job.
    +
    +The hierarchical orders are as follows:
    +
    +jm < jm.job 
    +
    +tm < tm.job < tm.task < tm.operator
    +
    +This hierarchy also defines which variables may be accessed. The `tm.operator` format may contain variables for jobs, whereas `tm.job` may not contain
    +variables for operators. There is no restriction on order of variables.
    +
    +There is no restriction on the order of variables.
    +
    +The default scope for operator metrics will result in a metric name akin to `localhost.taskmanager.1234.MyJob.MyOperator.0`
    +
    +If you also want to include the task name, but omit the taskmanager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: \<host>.\<job_name>.\<task_name>.\<operator_name>.\<subtask_index>`
    +
    +This could create the name `localhost.MyJob.MySource_->_MyOperator.MyOperator.0`.
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +### Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in the `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS
    +
    +You can write your own Reporter by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    --- End diff --
    
    Either putting `Reporter` in backticks or using a small letter would be better.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by uce <gi...@git.apache.org>.
Github user uce commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69163770
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    +      @Override
    +      public Integer getValue() {
    +        return valueToExpose;
    +      }});
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Histogram
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.histogram = getRuntimeContext().getMetricGroup().histogram("myHistogram", new MyHistogram());
    +    ...
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +    ...
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    --- End diff --
    
    I would rephrase this along the lines of "Flink does not provide a default implementation for `Histogram`, but offers a wrapper that allows usage of Dropwizard histograms (see LINK TO CASS ON GITHUB).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by uce <gi...@git.apache.org>.
Github user uce commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69166838
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    +      @Override
    +      public Integer getValue() {
    +        return valueToExpose;
    +      }});
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Histogram
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.histogram = getRuntimeContext().getMetricGroup().histogram("myHistogram", new MyHistogram());
    +    ...
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +    ...
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +This wrapper is contained in the `flink-metrics-dropwizard` module.
    +
    +### Scope
    --- End diff --
    
    I was wondering whether the names are case sensitive or not?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2158: [FLINK-4116] Metrics documentation

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on the issue:

    https://github.com/apache/flink/pull/2158
  
    also missing: specific reporter naming


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on the issue:

    https://github.com/apache/flink/pull/2158
  
    The documentation has reached a really good state. I had only some minor comments left.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68564162
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter() call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +- `metrics.scope.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +- `metrics.scope.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +There is no restriction on the order of variables.
    +
    +Which format is applied to a metric depends on which entities it was scoped to.
    +The `metrics.scope.operator` format will be applied to all metrics that were scoped to an Operator.
    +The `metrics.scope.tm.job` format will be applied to all metrics that were only scoped to a Job and a TaskManager.
    +
    +This allows you to define explicitly how metrics on every level should be reported.
    +
    +### Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in the `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter)
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010)
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS)
    +
    +By default Flink uses JMX to expose metrics.
    +
    +The port for JMX can be configured by setting the `metrics.jmx.port` key. This parameter expects either a single port
    +or a port range, with the default being 9010-9025. Which port is used in the end is shown in the relevant Job-/TaskManager log.
    +
    +All non-JMXReporters are not part of the distribution and have to be added to the classpath manually. (usually by putting the jar into /lib)
    +
    +You can write your own Reporter by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the Scheduled interface as well.
    +
    +Flink supports the following systems with their respective configuration parameters:
    +
    +#### Ganglia (org.apcahe.flink.metrics.ganglia.GangliaReporter)
    +- `host`
    +- `port`
    +- `dmax`
    +- `tmax`
    +- `ttl`
    +- `addressingMode` (UNICAST/MULTICAST)
    --- End diff --
    
    Explanation for all of the parameters required.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68910582
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter() call. The order is always \<system_scope>\<user_scope>\<name>.
    --- End diff --
    
    I don't understand how backticks solve the issue i mentioned.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69173467
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    +      @Override
    +      public Integer getValue() {
    +        return valueToExpose;
    +      }});
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Histogram
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.histogram = getRuntimeContext().getMetricGroup().histogram("myHistogram", new MyHistogram());
    +    ...
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +    ...
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +This wrapper is contained in the `flink-metrics-dropwizard` module.
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represents the entities it is tied to. By default a metric that is registered in a user function will be scoped to the operator in which the function runs, the task/job it belongs to and the taskManager/host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter()` call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +  - Applied to all metrics that were scoped to a jobmanager.
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +  - Applied to all metrics that were scoped to a jobmanager and job.
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +  - Applied to all metrics that were scoped to a taskmanager.
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +  - Applied to all metrics that were scoped to a taskmanager and job.
    +- `metrics.scope.tm.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +  - Applied to all metrics that were scoped to an operator.
    +
    +Note that for metrics for which multiple formats may apply (like jm and jm.job) the most specific format takes precedence,
    +in this case jm.job.
    +
    +The hierarchical orders are as follows:
    +
    +jm < jm.job 
    +
    +tm < tm.job < tm.task < tm.operator
    +
    +This hierarchy also defines which variables may be accessed. The `tm.operator` format may contain variables for jobs, whereas `tm.job` may not contain
    +variables for operators. There is no restriction on order of variables.
    +
    +There is no restriction on the order of variables.
    +
    +The default scope for operator metrics will result in a metric name akin to `localhost.taskmanager.1234.MyJob.MyOperator.0`
    +
    +If you also want to include the task name, but omit the taskmanager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: \<host>.\<job_name>.\<task_name>.\<operator_name>.\<subtask_index>`
    +
    +This could create the name `localhost.MyJob.MySource_->_MyOperator.MyOperator.0`.
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +### Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in the `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS
    +
    +You can write your own `Reporter` by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the `Scheduled` interface as well.
    --- End diff --
    
    JMX for example is not scheduled. I could also imagine Reporters that are queried from the outside.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2158: [FLINK-4116] Metrics documentation

Posted by uce <gi...@git.apache.org>.
Github user uce commented on the issue:

    https://github.com/apache/flink/pull/2158
  
    Good job Chesnay! Reads very well. I like the way the text and code examples are mixed. I had some minor inline comments.
    
    My biggest concern (not reflected in the inline comments) is the `Scope` section, which is too dense in my opinion. Do you think we can improve it somehow? 
    
    Maybe add sub sections `System Scope`, `User Scope`, `Examples`, `Variables` for starters. The `hierarchical orders` stuff I did not get.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68981861
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    +      @Override
    +      public Integer getValue() {
    +        return valueToExpose;
    +      }});
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Histogram
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.histogram = getRuntimeContext().getMetricGroup().histogram("myHistogram", new MyHistogram());
    +    ...
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +    ...
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +This wrapper is contained in the `flink-metrics-dropwizard` module.
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represents the entities it is tied to. By default a metric that is registered in a user function will be scoped to the operator in which the function runs, the task/job it belongs to and the taskManager/host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter()` call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +  - Applied to all metrics that were scoped to a jobmanager.
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +  - Applied to all metrics that were scoped to a jobmanager and job.
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +  - Applied to all metrics that were scoped to a taskmanager.
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +  - Applied to all metrics that were scoped to a taskmanager and job.
    +- `metrics.scope.tm.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +  - Applied to all metrics that were scoped to an operator.
    +
    +Note that for metrics for which multiple formats may apply (like jm and jm.job) the most specific format takes precedence,
    +in this case jm.job.
    +
    +The hierarchical orders are as follows:
    +
    +jm < jm.job 
    +
    +tm < tm.job < tm.task < tm.operator
    +
    +This hierarchy also defines which variables may be accessed. The `tm.operator` format may contain variables for jobs, whereas `tm.job` may not contain
    +variables for operators. There is no restriction on order of variables.
    +
    +There is no restriction on the order of variables.
    +
    +The default scope for operator metrics will result in a metric name akin to `localhost.taskmanager.1234.MyJob.MyOperator.0`
    +
    +If you also want to include the task name, but omit the taskmanager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: \<host>.\<job_name>.\<task_name>.\<operator_name>.\<subtask_index>`
    +
    +This could create the name `localhost.MyJob.MySource_->_MyOperator.MyOperator.0`.
    +
    +The following is a list of all variables available:
    +
    +- JobManager: \<host>
    +- TaskManager: \<host>, \<tm_id>
    +- Job: \<job_id>, \<job_name>
    +- Task: \<task_id>, \<task_name>, \<task_attempt_id>, \<task_attempt_num>, \<subtask_index>
    +- Operator: \<operator_name>, \<subtask_index>
    +
    +### Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in the `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS
    +
    +You can write your own Reporter by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the Scheduled interface as well.
    --- End diff --
    
    Same applies to `Scheduled`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r70097270
  
    --- Diff: docs/apis/metrics.md ---
    @@ -0,0 +1,441 @@
    +---
    +title: "Metrics"
    +# Top-level navigation
    +top-nav-group: apis
    +top-nav-pos: 13
    +top-nav-title: "Metrics"
    +---
    +<!--
    +Licensed to the Apache Software Foundation (ASF) under one
    +or more contributor license agreements.  See the NOTICE file
    +distributed with this work for additional information
    +regarding copyright ownership.  The ASF licenses this file
    +to you under the Apache License, Version 2.0 (the
    +"License"); you may not use this file except in compliance
    +with the License.  You may obtain a copy of the License at
    +
    +  http://www.apache.org/licenses/LICENSE-2.0
    +
    +Unless required by applicable law or agreed to in writing,
    +software distributed under the License is distributed on an
    +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    +KIND, either express or implied.  See the License for the
    +specific language governing permissions and limitations
    +under the License.
    +-->
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +* This will be replaced by the TOC
    +{:toc}
    +
    +## Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.counter = getRuntimeContext()
    +      .getMetricGroup()
    +      .counter("myCounter");
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    this.counter.inc();
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.counter = getRuntimeContext()
    +      .getMetricGroup()
    +      .counter("myCustomCounter", new CustomCounter());
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is no restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    getRuntimeContext()
    +      .getMetricGroup()
    +      .gauge("MyGauge", new Gauge<Integer>() {
    +        @Override
    +        public Integer getValue() {
    +          return valueToExpose;
    +        }
    +      });
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Note that reporters will turn the exposed object into a `String`, which means that a meaningful `toString()` implementation is required.
    +
    +#### Histogram
    +
    +A `Histogram` measures the distribution of long values.
    +You can register one by calling `histogram(String name, Histogram histogram)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.histogram = getRuntimeContext()
    +      .getMetricGroup()
    +      .histogram("myHistogram", new MyHistogram());
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink does not provide a default implementation for `Histogram`, but offers a {% gh_link flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/DropwizardHistogramWrapper.java "Wrapper" %} that allows usage of Codahale/DropWizard histograms.
    +To use this wrapper add the following dependency in your `pom.xml`:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-dropwizard</artifactId>
    +      <version>{{site.version}}</version>
    +</dependency>
    +{% endhighlight %}
    +
    +You can then register a Codahale/DropWizard histogram like this:
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    com.codahale.metrics.Histogram histogram =
    +      new com.codahale.metrics.Histogram(new SlidingWindowReservoir(500));
    +
    +    this.histogram = getRuntimeContext()
    +      .getMetricGroup()
    +      .histogram("myHistogram", new DropWizardHistogramWrapper(histogram));
    +  }
    +}
    +{% endhighlight %}
    +
    +## Scope
    +
    +Every metric is assigned an identifier under which it will be reported that is based on 3 components: the user-provided name when registering the metric, an optional user-defined scope and a system-provided scope.
    +For example, if `A.B` is the sytem scope, `C.D` the user scope and `E` the name, then the identifier for the metric will be `A.B.C.D.E`.
    +
    +### User Scope
    +
    +You can define a user scope by calling either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext()
    +  .getMetricGroup()
    +  .addGroup("MyMetrics")
    +  .counter("myCounter");
    +
    +{% endhighlight %}
    +
    +### System Scope
    +
    +The system scope contains context information about the metric, for example in which task it was registered or what job that task belongs to.
    +
    +Which context information should be included can be configured by setting the following keys in `conf/flink-conf.yaml`.
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "&lt;task_id&gt;") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: &lt;host&gt;.jobmanager
    +  - Applied to all metrics that were scoped to a job manager.
    +- `metrics.scope.jm.job`
    +  - Default: &lt;host&gt;.jobmanager.&lt;job_name&gt;
    +  - Applied to all metrics that were scoped to a job manager and job.
    +- `metrics.scope.tm`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;
    +  - Applied to all metrics that were scoped to a task manager.
    +- `metrics.scope.tm.job`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;
    +  - Applied to all metrics that were scoped to a task manager and job.
    +- `metrics.scope.tm.task`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;.&lt;task_name&gt;.&lt;subtask_index&gt;
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;.&lt;operator_name&gt;.&lt;subtask_index&gt;
    +  - Applied to all metrics that were scoped to an operator.
    +
    +There are no restrictions on the number or order of variables. Variables are case sensitive.
    +
    +The default scope for operator metrics will result in an identifier akin to `localhost.taskmanager.1234.MyJob.MyOperator.0.MyMetric`
    +
    +If you also want to include the task name but omit the task manager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: <host>.<job_name>.<task_name>.<operator_name>.<subtask_index>`
    +
    +This could create the identifier `localhost.MyJob.MySource_->_MyOperator.MyOperator.0.MyMetric`.
    +
    +Note that for this format string an identifier clash can occur should the same job be run multiple times concurrently, which can lead to inconsistent metric data.
    +As such it is advised to either use format strings that provide a certain degree of uniqueness by including IDs (e.g &lt;job_id&gt;)
    +or by assigning unique names to jobs and operators.
    +
    +### List of all Variables
    +
    +- JobManager: &lt;host&gt;
    +- TaskManager: &lt;host&gt;, &lt;tm_id&gt;
    +- Job: &lt;job_id&gt;, &lt;job_name&gt;
    +- Task: &lt;task_id&gt;, &lt;task_name&gt;, &lt;task_attempt_id&gt;, &lt;task_attempt_num&gt;, &lt;subtask_index&gt;
    +- Operator: &lt;operator_name&gt;, &lt;subtask_index&gt;
    +
    +## Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS
    +
    +You can write your own `Reporter` by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the `Scheduled` interface as well.
    +
    +By default Flink uses JMX to expose metrics.
    +All non-JMXReporters are not part of the distribution. To use them you have to copy the respective fat jar to the `/lib` folder.
    +
    +The following sections list the supported reporters.
    +
    +### JMX
    +
    +The port for JMX can be configured by setting the `metrics.jmx.port` key. This parameter expects either a single port
    +or a port range, with the default being 9010-9025. The used port is shown in the relevant job or task manager log.
    +
    +### Ganglia (org.apache.flink.metrics.ganglia.GangliaReporter)
    +Dependency:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-ganglia</artifactId>
    +      <version>{{site.version}}</version>
    +</dependency>
    +{% endhighlight %}
    +
    +Parameters:
    +
    +- `host` - the ganglia server host
    --- End diff --
    
    We should highlight that this is the value you set in `gmond.conf` `udp_recv_channel.bind`. Otherwise the UPD packages won't arrive. That was actually the problem I had the whole time. I'm just not good with tooling...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68562873
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    --- End diff --
    
    Maybe we could also say which module contains the `DropWizardHistogramWrapper`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by uce <gi...@git.apache.org>.
Github user uce commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69160791
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the MetricGroup#addGroup((int/String) name) method.
    +
    +{% highlight java %}
    +
    +counter2 = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter2");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter() call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    --- End diff --
    
    I agree, but note that there are other components who also do it outside of `ConfigConstants`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by uce <gi...@git.apache.org>.
Github user uce commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69162378
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    --- End diff --
    
    the line breaks in my browser are bad, maybe move everything after `getMetricGroup()` to the next line? Furthermore, `getMetricGroup` has a lower letter `m`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69173815
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    +      @Override
    +      public Integer getValue() {
    +        return valueToExpose;
    +      }});
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Histogram
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.histogram = getRuntimeContext().getMetricGroup().histogram("myHistogram", new MyHistogram());
    +    ...
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +    ...
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +This wrapper is contained in the `flink-metrics-dropwizard` module.
    +
    +### Scope
    --- End diff --
    
    the variable names? yes


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68563031
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,211 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows users gather and expose metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling getRuntimeContext().getMetricGroup().
    +This method returns a MetricGroup object on which you can create and register new metrics.
    +
    +If you want to count the number of records your user function has received you could use a Counter like this:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +### Metric types
    +
    +Flink supports Counters, Gauges and Histograms.
    +
    +A Counter is used to count something. The current value can be in- or decremented using `inc([long n])` or `dec([long n])`.
    +You can create and registers a Counter by calling `counter(String name)` on a MetricGroup. Alternatively, you can pass an instance of your own Counter implementation along with the name.
    +
    +A Gauge provides a value of any type on demand. In order to use a Gauge you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represent the entities it is tied to. By default a metric that is registered in a user function will be scoped to the Operator in which the function runs, the Task/Job it belongs to and the TaskManager/Host it is executed on. This is referred to as the "system scope".
    --- End diff --
    
    task/job with small letter


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69163312
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    --- End diff --
    
    no, reporters generally use toString() on the value returned. that should probably be documented though.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r68981623
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    +      @Override
    +      public Integer getValue() {
    +        return valueToExpose;
    +      }});
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Histogram
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.histogram = getRuntimeContext().getMetricGroup().histogram("myHistogram", new MyHistogram());
    +    ...
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +    ...
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink only provides an interface for Histograms, but offers a Wrapper that allows usage of Codahale/DropWizard Histograms. (org.apache.flink.dropwizard.metrics.DropWizardHistogramWrapper)
    +This wrapper is contained in the `flink-metrics-dropwizard` module.
    +
    +### Scope
    +
    +Every registered metric has an automatically assigned scope which represents the entities it is tied to. By default a metric that is registered in a user function will be scoped to the operator in which the function runs, the task/job it belongs to and the taskManager/host it is executed on. This is referred to as the "system scope".
    +
    +You can define an additonal "user scope" by calling the either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext().getMetricGroup().addGroup("MyMetrics").counter("myCounter");
    +
    +{% endhighlight %}
    +
    +The name under which a metric is exported is based on both scopes and the name passed in the `counter()` call. The order is always \<system_scope>\<user_scope>\<name>.
    +
    +The system scope allows the reported name to contain contextual information like the name of job it was registered in without requiring the user to pass this information manually.
    +
    +How the system scope affects the reported name for a metric can be modified by setting the following keys in the flink-conf.yaml. 
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "\<task_id>") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: \<host>.jobmanager
    +  - Applied to all metrics that were scoped to a jobmanager.
    +- `metrics.scope.jm.job`
    +  - Default: \<host>.jobmanager.\<job_name>
    +  - Applied to all metrics that were scoped to a jobmanager and job.
    +- `metrics.scope.tm`
    +  - Default: \<host>.taskmanager.\<tm_id>
    +  - Applied to all metrics that were scoped to a taskmanager.
    +- `metrics.scope.tm.job`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>
    +  - Applied to all metrics that were scoped to a taskmanager and job.
    +- `metrics.scope.tm.task`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<task_name>.\<subtask_index>
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: \<host>.taskmanager.\<tm_id>.\<job_name>.\<operator_name>.\<subtask_index>
    +  - Applied to all metrics that were scoped to an operator.
    +
    +Note that for metrics for which multiple formats may apply (like jm and jm.job) the most specific format takes precedence,
    +in this case jm.job.
    +
    +The hierarchical orders are as follows:
    +
    +jm < jm.job 
    +
    +tm < tm.job < tm.task < tm.operator
    +
    +This hierarchy also defines which variables may be accessed. The `tm.operator` format may contain variables for jobs, whereas `tm.job` may not contain
    +variables for operators. There is no restriction on order of variables.
    +
    +There is no restriction on the order of variables.
    +
    +The default scope for operator metrics will result in a metric name akin to `localhost.taskmanager.1234.MyJob.MyOperator.0`
    +
    +If you also want to include the task name, but omit the taskmanager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: \<host>.\<job_name>.\<task_name>.\<operator_name>.\<subtask_index>`
    --- End diff --
    
    Do we need the `\` in front of the variables? They are also shown in the rendered html document.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69163084
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    --- End diff --
    
    I'm not too happy with the names either.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r70044632
  
    --- Diff: docs/apis/metrics.md ---
    @@ -0,0 +1,441 @@
    +---
    +title: "Metrics"
    +# Top-level navigation
    +top-nav-group: apis
    +top-nav-pos: 13
    +top-nav-title: "Metrics"
    +---
    +<!--
    +Licensed to the Apache Software Foundation (ASF) under one
    +or more contributor license agreements.  See the NOTICE file
    +distributed with this work for additional information
    +regarding copyright ownership.  The ASF licenses this file
    +to you under the Apache License, Version 2.0 (the
    +"License"); you may not use this file except in compliance
    +with the License.  You may obtain a copy of the License at
    +
    +  http://www.apache.org/licenses/LICENSE-2.0
    +
    +Unless required by applicable law or agreed to in writing,
    +software distributed under the License is distributed on an
    +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    +KIND, either express or implied.  See the License for the
    +specific language governing permissions and limitations
    +under the License.
    +-->
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +* This will be replaced by the TOC
    +{:toc}
    +
    +## Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.counter = getRuntimeContext()
    +      .getMetricGroup()
    +      .counter("myCounter");
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    this.counter.inc();
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.counter = getRuntimeContext()
    +      .getMetricGroup()
    +      .counter("myCustomCounter", new CustomCounter());
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is no restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    getRuntimeContext()
    +      .getMetricGroup()
    +      .gauge("MyGauge", new Gauge<Integer>() {
    +        @Override
    +        public Integer getValue() {
    +          return valueToExpose;
    +        }
    +      });
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Note that reporters will turn the exposed object into a `String`, which means that a meaningful `toString()` implementation is required.
    +
    +#### Histogram
    +
    +A `Histogram` measures the distribution of long values.
    +You can register one by calling `histogram(String name, Histogram histogram)` on a `MetricGroup`.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    this.histogram = getRuntimeContext()
    +      .getMetricGroup()
    +      .histogram("myHistogram", new MyHistogram());
    +  }
    +
    +  @public Integer map(Long value) throws Exception {
    +    this.histogram.update(value);
    +  }
    +}
    +{% endhighlight %}
    +
    +Flink does not provide a default implementation for `Histogram`, but offers a {% gh_link flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/DropwizardHistogramWrapper.java "Wrapper" %} that allows usage of Codahale/DropWizard histograms.
    +To use this wrapper add the following dependency in your `pom.xml`:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-dropwizard</artifactId>
    +      <version>{{site.version}}</version>
    +</dependency>
    +{% endhighlight %}
    +
    +You can then register a Codahale/DropWizard histogram like this:
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    com.codahale.metrics.Histogram histogram =
    +      new com.codahale.metrics.Histogram(new SlidingWindowReservoir(500));
    +
    +    this.histogram = getRuntimeContext()
    +      .getMetricGroup()
    +      .histogram("myHistogram", new DropWizardHistogramWrapper(histogram));
    +  }
    +}
    +{% endhighlight %}
    +
    +## Scope
    +
    +Every metric is assigned an identifier under which it will be reported that is based on 3 components: the user-provided name when registering the metric, an optional user-defined scope and a system-provided scope.
    +For example, if `A.B` is the sytem scope, `C.D` the user scope and `E` the name, then the identifier for the metric will be `A.B.C.D.E`.
    +
    +### User Scope
    +
    +You can define a user scope by calling either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`.
    +
    +{% highlight java %}
    +
    +counter = getRuntimeContext()
    +  .getMetricGroup()
    +  .addGroup("MyMetrics")
    +  .counter("myCounter");
    +
    +{% endhighlight %}
    +
    +### System Scope
    +
    +The system scope contains context information about the metric, for example in which task it was registered or what job that task belongs to.
    +
    +Which context information should be included can be configured by setting the following keys in `conf/flink-conf.yaml`.
    +Each of these keys expect a format string that may contain constants (e.g. "taskmanager") and variables (e.g. "&lt;task_id&gt;") which will be replaced at runtime.
    +
    +- `metrics.scope.jm`
    +  - Default: &lt;host&gt;.jobmanager
    +  - Applied to all metrics that were scoped to a job manager.
    +- `metrics.scope.jm.job`
    +  - Default: &lt;host&gt;.jobmanager.&lt;job_name&gt;
    +  - Applied to all metrics that were scoped to a job manager and job.
    +- `metrics.scope.tm`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;
    +  - Applied to all metrics that were scoped to a task manager.
    +- `metrics.scope.tm.job`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;
    +  - Applied to all metrics that were scoped to a task manager and job.
    +- `metrics.scope.tm.task`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;.&lt;task_name&gt;.&lt;subtask_index&gt;
    +   - Applied to all metrics that were scoped to a task.
    +- `metrics.scope.tm.operator`
    +  - Default: &lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;.&lt;operator_name&gt;.&lt;subtask_index&gt;
    +  - Applied to all metrics that were scoped to an operator.
    +
    +There are no restrictions on the number or order of variables. Variables are case sensitive.
    +
    +The default scope for operator metrics will result in an identifier akin to `localhost.taskmanager.1234.MyJob.MyOperator.0.MyMetric`
    +
    +If you also want to include the task name but omit the task manager information you can specify the following format:
    +
    +`metrics.scope.tm.operator: <host>.<job_name>.<task_name>.<operator_name>.<subtask_index>`
    +
    +This could create the identifier `localhost.MyJob.MySource_->_MyOperator.MyOperator.0.MyMetric`.
    +
    +Note that for this format string an identifier clash can occur should the same job be run multiple times concurrently, which can lead to inconsistent metric data.
    +As such it is advised to either use format strings that provide a certain degree of uniqueness by including IDs (e.g &lt;job_id&gt;)
    +or by assigning unique names to jobs and operators.
    +
    +### List of all Variables
    +
    +- JobManager: &lt;host&gt;
    +- TaskManager: &lt;host&gt;, &lt;tm_id&gt;
    +- Job: &lt;job_id&gt;, &lt;job_name&gt;
    +- Task: &lt;task_id&gt;, &lt;task_name&gt;, &lt;task_attempt_id&gt;, &lt;task_attempt_num&gt;, &lt;subtask_index&gt;
    +- Operator: &lt;operator_name&gt;, &lt;subtask_index&gt;
    +
    +## Reporter
    +
    +Metrics can be exposed to an external system by configuring a reporter in `conf/flink-conf.yaml`.
    +
    +- `metrics.reporter.class`: The class of the reporter to use.
    +  - Example: org.apache.flink.metrics.reporter.JMXReporter
    +- `metrics.reporter.arguments`: A list of named parameters that are passed to the reporter.
    +  - Example: --host localhost --port 9010
    +- `metrics.reporter.interval`: The interval between reports.
    +  - Example: 10 SECONDS
    +
    +You can write your own `Reporter` by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface.
    +If the Reporter should send out reports regularly you have to implement the `Scheduled` interface as well.
    +
    +By default Flink uses JMX to expose metrics.
    +All non-JMXReporters are not part of the distribution and have to be added [manually]({{site.baseurl}}/apis/cluster_execution.html#linking-with-modules-not-contained-in-the-binary-distribution).
    +
    +The following sections list the supported reporters.
    +
    +### JMX
    +
    +The port for JMX can be configured by setting the `metrics.jmx.port` key. This parameter expects either a single port
    +or a port range, with the default being 9010-9025. The used port is shown in the relevant job or task manager log.
    +
    +### Ganglia (org.apache.flink.metrics.ganglia.GangliaReporter)
    +Dependency:
    +{% highlight xml %}
    +<dependency>
    +      <groupId>org.apache.flink</groupId>
    +      <artifactId>flink-metrics-ganglia</artifactId>
    +      <version>{{site.version}}</version>
    +</dependency>
    --- End diff --
    
    right, of course. I've adjusted the docs.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2158: [FLINK-4116] Metrics documentation

Posted by uce <gi...@git.apache.org>.
Github user uce commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2158#discussion_r69163196
  
    --- Diff: docs/apis/common/index.md ---
    @@ -1350,3 +1350,397 @@ You may specify program arguments before the job is executed. The plan visualiza
     the execution plan before executing the Flink job.
     
     {% top %}
    +
    +Metrics
    +-------------------
    +
    +Flink exposes a metric system that allows gathering and exposing metrics to external systems.
    +
    +### Registering metrics
    +
    +You can access the metric system from any user function that extends [RichFunction]({{ site.baseurl }}/apis/common/index.html#rich-functions) by calling `getRuntimeContext().getMetricGroup()`.
    +This method returns a `MetricGroup` object on which you can create and register new metrics.
    +
    +### Metric types
    +
    +Flink supports `Counters`, `Gauges` and `Histograms`.
    +
    +#### Counter
    +
    +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`.
    +You can create and register a `Counter` by calling `counter(String name)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private Counter counter;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.counter = getRuntimeContext().getMetricGroup().counter("myCounter");
    +    ...
    +  }
    +
    +  @public Integer map(String value) throws Exception {
    +    // increment counter
    +    this.counter.inc();
    +    ...
    +  }
    +}
    +
    +{% endhighlight %}
    +
    +Alternatively you can also use your own `Counter` implementation:
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  ...
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register a custom counter
    +    this.counter = getRuntimeContext().getmetricGroup().counter("myCustomCounter", new CustomCounter());
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Gauge
    +
    +A `Gauge` provides a value of any type on demand. In order to use a `Gauge` you must first create a class that implements the `org.apache.flink.metrics.Gauge` interface.
    +There is not restriction for the type of the returned value.
    +You can register a gauge by calling `gauge(String name, Gauge gauge)` on a MetricGroup.
    +
    +{% highlight java %}
    +
    +public class MyMapper extends RichMapFunction<String, Integer> {
    +  private int valueToExpose;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // register the gauge
    +    getRuntimeContext().getmetricGroup().gauge("MyGauge", new Gauge<Integer>() {
    +      @Override
    +      public Integer getValue() {
    +        return valueToExpose;
    +      }});
    +    ...
    +  }
    +  ...
    +}
    +
    +{% endhighlight %}
    +
    +#### Histogram
    +
    +A Histogram measure the distribution of long values.
    +You can register one by calling histogram(String name, Histogram histogram) on a MetricGroup.
    +
    +{% highlight java %}
    +public class MyMapper extends RichMapFunction<Long, Integer> {
    +  private Histogram histogram;
    +
    +  @Override
    +  public void open(Configuration config) {
    +    // create and register a counter
    +    this.histogram = getRuntimeContext().getMetricGroup().histogram("myHistogram", new MyHistogram());
    --- End diff --
    
    same line break issue as the custom counter


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---