[ 
https://issues.apache.org/jira/browse/FLINK-4116?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15367873#comment-15367873
 ] 

ASF GitHub Bot commented on FLINK-4116:
---------------------------------------

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...


> Document metrics
> ----------------
>
>                 Key: FLINK-4116
>                 URL: https://issues.apache.org/jira/browse/FLINK-4116
>             Project: Flink
>          Issue Type: Improvement
>          Components: Documentation, Metrics
>    Affects Versions: 1.1.0
>            Reporter: Chesnay Schepler
>            Assignee: Chesnay Schepler
>             Fix For: 1.1.0
>
>
> The metric system is currently not documented, which should be fixed before 
> the 1.1 release.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to