Skip to content

Commit

Permalink
[FLINK-4831][metrics] Implement slf4j metric reporter
Browse files Browse the repository at this point in the history
This closes apache#4661.
  • Loading branch information
yew1eb authored and zentol committed Oct 26, 2017
1 parent b7f0f5f commit c0199f5
Show file tree
Hide file tree
Showing 9 changed files with 531 additions and 0 deletions.
16 changes: 16 additions & 0 deletions docs/monitoring/metrics.md
Original file line number Diff line number Diff line change
Expand Up @@ -491,6 +491,22 @@ metrics.reporter.dghttp.tags: myflinkapp,prod

{% endhighlight %}


### Slf4j (org.apache.flink.metrics.slf4j.Slf4jReporter)

In order to use this reporter you must copy `/opt/flink-metrics-slf4j-{{site.version}}.jar` into the `/lib` folder
of your Flink distribution.

Example configuration:

{% highlight yaml %}

metrics.reporters: slf4j
metrics.reporter.slf4j.class: org.apache.flink.metrics.slf4j.Slf4jReporter
metrics.reporter.slf4j.interval: 60 SECONDS

{% endhighlight %}

## System metrics

By default Flink gathers several metrics that provide deep insights on the current state.
Expand Down
7 changes: 7 additions & 0 deletions flink-dist/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -253,6 +253,13 @@ under the License.
<version>${project.version}</version>
<scope>provided</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-metrics-slf4j</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
<!-- end optional Flink metrics reporters -->

<!-- start optional Flink libraries -->
Expand Down
7 changes: 7 additions & 0 deletions flink-dist/src/main/assemblies/opt.xml
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,13 @@
<fileMode>0644</fileMode>
</file>

<file>
<source>../flink-metrics/flink-metrics-slf4j/target/flink-metrics-slf4j-${project.version}.jar</source>
<outputDirectory>opt/</outputDirectory>
<destName>flink-metrics-slf4j-${project.version}.jar</destName>
<fileMode>0644</fileMode>
</file>

<file>
<source>../flink-filesystems/flink-s3-fs-hadoop/target/flink-s3-fs-hadoop-${project.version}.jar</source>
<outputDirectory>opt/</outputDirectory>
Expand Down
82 changes: 82 additions & 0 deletions flink-metrics/flink-metrics-slf4j/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,82 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
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:https://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.
-->
<project xmlns="http:https://maven.apache.org/POM/4.0.0" xmlns:xsi="http:https://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http:https://maven.apache.org/POM/4.0.0 http:https://maven.apache.org/maven-v4_0_0.xsd">

<modelVersion>4.0.0</modelVersion>

<parent>
<groupId>org.apache.flink</groupId>
<artifactId>flink-metrics</artifactId>
<version>1.4-SNAPSHOT</version>
<relativePath>..</relativePath>
</parent>

<artifactId>flink-metrics-slf4j</artifactId>
<name>flink-metrics-slf4j</name>

<dependencies>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-annotations</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-metrics-core</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>

<!-- test dependencies -->

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-core</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-runtime_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-runtime_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>

</dependencies>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,143 @@
/*
* 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:https://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.
*/

package org.apache.flink.metrics.slf4j;

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.metrics.Counter;
import org.apache.flink.metrics.Gauge;
import org.apache.flink.metrics.Histogram;
import org.apache.flink.metrics.HistogramStatistics;
import org.apache.flink.metrics.Meter;
import org.apache.flink.metrics.Metric;
import org.apache.flink.metrics.MetricConfig;
import org.apache.flink.metrics.reporter.AbstractReporter;
import org.apache.flink.metrics.reporter.MetricReporter;
import org.apache.flink.metrics.reporter.Scheduled;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.Map;

/**
* {@link MetricReporter} that exports {@link Metric Metrics} via SLF4J {@link Logger}.
*/
public class Slf4jReporter extends AbstractReporter implements Scheduled {
private static final Logger LOG = LoggerFactory.getLogger(Slf4jReporter.class);
private static final String lineSeparator = System.lineSeparator();

@VisibleForTesting
public Map<Gauge<?>, String> getGauges() {
return gauges;
}

@VisibleForTesting
public Map<Counter, String> getCounters() {
return counters;
}

@VisibleForTesting
public Map<Histogram, String> getHistograms() {
return histograms;
}

@VisibleForTesting
public Map<Meter, String> getMeters() {
return meters;
}

@Override
public void open(MetricConfig metricConfig) {
}

@Override
public void close() {
}

@Override
public void report() {
StringBuilder builder = new StringBuilder();
builder
.append(lineSeparator)
.append("=========================== Starting metrics report ===========================")
.append(lineSeparator);

builder
.append(lineSeparator)
.append("-- Counters -------------------------------------------------------------------")
.append(lineSeparator);
for (Map.Entry<Counter, String> metric : counters.entrySet()) {
builder
.append(metric.getValue()).append(": ").append(metric.getKey().getCount())
.append(lineSeparator);
}

builder
.append(lineSeparator)
.append("-- Gauges ---------------------------------------------------------------------")
.append(lineSeparator);
for (Map.Entry<Gauge<?>, String> metric : gauges.entrySet()) {
builder
.append(metric.getValue()).append(": ").append(metric.getKey().getValue())
.append(lineSeparator);
}

builder
.append(lineSeparator)
.append("-- Meters ---------------------------------------------------------------------")
.append(lineSeparator);
for (Map.Entry<Meter, String> metric : meters.entrySet()) {
builder
.append(metric.getValue()).append(": ").append(metric.getKey().getRate())
.append(lineSeparator);
}

builder
.append(lineSeparator)
.append("-- Histograms -----------------------------------------------------------------")
.append(lineSeparator);
for (Map.Entry<Histogram, String> metric : histograms.entrySet()) {
HistogramStatistics stats = metric.getKey().getStatistics();
builder
.append(metric.getValue()).append(": count=").append(stats.size())
.append(", min=").append(stats.getMin())
.append(", max=").append(stats.getMax())
.append(", mean=").append(stats.getMean())
.append(", stddev=").append(stats.getStdDev())
.append(", p50=").append(stats.getQuantile(0.50))
.append(", p75=").append(stats.getQuantile(0.75))
.append(", p95=").append(stats.getQuantile(0.95))
.append(", p98=").append(stats.getQuantile(0.98))
.append(", p99=").append(stats.getQuantile(0.99))
.append(", p999=").append(stats.getQuantile(0.999))
.append(lineSeparator);
}

builder
.append(lineSeparator)
.append("=========================== Finished metrics report ===========================")
.append(lineSeparator);
LOG.info(builder.toString());
}

@Override
public String filterCharacters(String input) {
return input;
}
}
Loading

0 comments on commit c0199f5

Please sign in to comment.