Skip to content

Commit

Permalink
[FLINK-17812][dist] Bundle reporters in plugins/ directory
Browse files Browse the repository at this point in the history
  • Loading branch information
zentol committed May 27, 2020
1 parent dcf4cd3 commit 3623967
Show file tree
Hide file tree
Showing 8 changed files with 64 additions and 89 deletions.
22 changes: 3 additions & 19 deletions docs/monitoring/metrics.md
Original file line number Diff line number Diff line change
Expand Up @@ -587,10 +587,12 @@ metrics.reporter.my_other_reporter.port: 10000

**Important:** The jar containing the reporter must be accessible when Flink is started. Reporters that support the
`factory.class` property can be loaded as [plugins]({{ site.baseurl }}/ops/plugins). Otherwise the jar must be placed
in the /lib folder.
in the /lib folder. Reporters that are shipped with Flink (i.e., all reporters documented on this page) are available
by default.

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 additionally implementing a `MetricReporterFactory` your reporter can also be loaded as a plugin.

The following sections list the supported reporters.

Expand Down Expand Up @@ -630,9 +632,6 @@ The domain thus identifies a metric class, while the key-property list identifie

### Graphite (org.apache.flink.metrics.graphite.GraphiteReporter)

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

Parameters:

- `host` - the Graphite server host
Expand Down Expand Up @@ -681,9 +680,6 @@ All Flink metrics variables (see [List of all Variables](#list-of-all-variables)

### Prometheus (org.apache.flink.metrics.prometheus.PrometheusReporter)

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

Parameters:

- `port` - (optional) the port the Prometheus exporter listens on, defaults to [9249](https://github.com/prometheus/prometheus/wiki/Default-port-allocations). In order to be able to run several instances of the reporter on one host (e.g. when one TaskManager is colocated with the JobManager) it is advisable to use a port range like `9250-9260`.
Expand All @@ -710,9 +706,6 @@ All Flink metrics variables (see [List of all Variables](#list-of-all-variables)

### PrometheusPushGateway (org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporter)

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

Parameters:

{% include generated/prometheus_push_gateway_reporter_configuration.html %}
Expand All @@ -737,9 +730,6 @@ Please see the [Prometheus documentation](https://prometheus.io/docs/practices/p

### StatsD (org.apache.flink.metrics.statsd.StatsDReporter)

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

Parameters:

- `host` - the StatsD server host
Expand All @@ -757,9 +747,6 @@ metrics.reporter.stsd.port: 8125

### Datadog (org.apache.flink.metrics.datadog.DatadogHttpReporter)

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

Note any variables in Flink metrics, such as `<host>`, `<job_name>`, `<tm_id>`, `<subtask_index>`, `<task_name>`, and `<operator_name>`,
will be sent to Datadog as tags. Tags will look like `host:localhost` and `job_name:myjobname`.

Expand Down Expand Up @@ -789,9 +776,6 @@ metrics.reporter.dhhttp.maxMetricsPerRequest: 2000

### 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 `/plugins/slf4j` folder
of your Flink distribution.

Example configuration:

{% highlight yaml %}
Expand Down
22 changes: 3 additions & 19 deletions docs/monitoring/metrics.zh.md
Original file line number Diff line number Diff line change
Expand Up @@ -587,10 +587,12 @@ metrics.reporter.my_other_reporter.port: 10000

**Important:** The jar containing the reporter must be accessible when Flink is started. Reporters that support the
`factory.class` property can be loaded as [plugins]({{ site.baseurl }}/ops/plugins). Otherwise the jar must be placed
in the /lib folder.
in the /lib folder. Reporters that are shipped with Flink (i.e., all reporters documented on this page) are available
by default.

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 additionally implementing a `MetricReporterFactory` your reporter can also be loaded as a plugin.

The following sections list the supported reporters.

Expand Down Expand Up @@ -630,9 +632,6 @@ The domain thus identifies a metric class, while the key-property list identifie

### Graphite (org.apache.flink.metrics.graphite.GraphiteReporter)

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

Parameters:

- `host` - the Graphite server host
Expand Down Expand Up @@ -681,9 +680,6 @@ All Flink metrics variables (see [List of all Variables](#list-of-all-variables)

### Prometheus (org.apache.flink.metrics.prometheus.PrometheusReporter)

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

Parameters:

- `port` - (optional) the port the Prometheus exporter listens on, defaults to [9249](https://github.com/prometheus/prometheus/wiki/Default-port-allocations). In order to be able to run several instances of the reporter on one host (e.g. when one TaskManager is colocated with the JobManager) it is advisable to use a port range like `9250-9260`.
Expand All @@ -710,9 +706,6 @@ All Flink metrics variables (see [List of all Variables](#list-of-all-variables)

### PrometheusPushGateway (org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporter)

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

Parameters:

{% include generated/prometheus_push_gateway_reporter_configuration.html %}
Expand All @@ -737,9 +730,6 @@ Please see the [Prometheus documentation](https://prometheus.io/docs/practices/p

### StatsD (org.apache.flink.metrics.statsd.StatsDReporter)

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

Parameters:

- `host` - the StatsD server host
Expand All @@ -757,9 +747,6 @@ metrics.reporter.stsd.port: 8125

### Datadog (org.apache.flink.metrics.datadog.DatadogHttpReporter)

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

Note any variables in Flink metrics, such as `<host>`, `<job_name>`, `<tm_id>`, `<subtask_index>`, `<task_name>`, and `<operator_name>`,
will be sent to Datadog as tags. Tags will look like `host:localhost` and `job_name:myjobname`.

Expand Down Expand Up @@ -789,9 +776,6 @@ metrics.reporter.dhhttp.maxMetricsPerRequest: 2000

### 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 `/plugins/slf4j` folder
of your Flink distribution.

Example configuration:

{% highlight yaml %}
Expand Down
7 changes: 6 additions & 1 deletion docs/ops/plugins.md
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ under the License.
Plugins facilitate a strict separation of code through restricted classloaders. Plugins cannot
access classes from other plugins or from Flink that have not been specifically whitelisted. This
strict isolation allows plugins to contain conflicting versions of the same library without the need
to relocate classes or to converge to common versions. Currently, only file systems are pluggable
to relocate classes or to converge to common versions. Currently, file systems and metric reporters are pluggable
but in the future, connectors, formats, and even user code should also be pluggable.

* This will be replaced by the TOC
Expand Down Expand Up @@ -114,4 +114,9 @@ The whitelisted classes mainly consists of the necessary interfaces to implement
Furthermore, loggers are whitelisted, so that they are configured properly.
-->

## Metric Reporters

All [metric reporters](../monitoring/metrics.html#Reporter) that Flink provides can be used as plugins.
See the [metrics](../monitoring/metrics.html) documentation for more details.

{% top %}
7 changes: 6 additions & 1 deletion docs/ops/plugins.zh.md
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ under the License.
Plugins facilitate a strict separation of code through restricted classloaders. Plugins cannot
access classes from other plugins or from Flink that have not been specifically whitelisted. This
strict isolation allows plugins to contain conflicting versions of the same library without the need
to relocate classes or to converge to common versions. Currently, only file systems are pluggable
to relocate classes or to converge to common versions. Currently, file systems and metric reporters are pluggable
but in the future, connectors, formats, and even user code should also be pluggable.

* This will be replaced by the TOC
Expand Down Expand Up @@ -114,4 +114,9 @@ The whitelisted classes mainly consists of the necessary interfaces to implement
Furthermore, loggers are whitelisted, so that they are configured properly.
-->

## Metric Reporters

All [metric reporters](../monitoring/metrics.html#Reporter) that Flink provides can be used as plugins.
See the [metrics](../monitoring/metrics.html) documentation for more details.

{% top %}
43 changes: 0 additions & 43 deletions flink-dist/src/main/assemblies/opt.xml
Original file line number Diff line number Diff line change
Expand Up @@ -97,49 +97,6 @@
<fileMode>0755</fileMode>
</file>

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

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

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

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

<file>
<source>../flink-metrics/flink-metrics-datadog/target/flink-metrics-datadog-${project.version}.jar</source>
<outputDirectory>opt/</outputDirectory>
<destName>flink-metrics-datadog-${project.version}.jar</destName>
<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
42 changes: 42 additions & 0 deletions flink-dist/src/main/assemblies/plugins.xml
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,48 @@
<destName>flink-metrics-jmx-${project.version}.jar</destName>
<fileMode>0644</fileMode>
</file>

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

<file>
<source>../flink-metrics/flink-metrics-influxdb/target/flink-metrics-influxdb_${scala.binary.version}-${project.version}.jar</source>
<outputDirectory>plugins/metrics_influx/</outputDirectory>
<destName>flink-metrics-influxdb-${project.version}.jar</destName>
<fileMode>0644</fileMode>
</file>

<file>
<source>../flink-metrics/flink-metrics-prometheus/target/flink-metrics-prometheus_${scala.binary.version}-${project.version}.jar</source>
<outputDirectory>plugins/metrics_prometheus/</outputDirectory>
<destName>flink-metrics-prometheus-${project.version}.jar</destName>
<fileMode>0644</fileMode>
</file>

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

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

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

</assembly>
Original file line number Diff line number Diff line change
Expand Up @@ -128,18 +128,17 @@ public static void checkOS() {
public static Collection<TestParams> testParameters() {
return Arrays.asList(
TestParams.from("Jar in 'lib'",
builder -> builder.copyJar(PROMETHEUS_JAR_PREFIX, JarLocation.OPT, JarLocation.LIB),
builder -> builder.moveJar(PROMETHEUS_JAR_PREFIX, JarLocation.PLUGINS, JarLocation.LIB),
REFLECTION),
TestParams.from("Jar in 'lib'",
builder -> builder.copyJar(PROMETHEUS_JAR_PREFIX, JarLocation.OPT, JarLocation.LIB),
builder -> builder.moveJar(PROMETHEUS_JAR_PREFIX, JarLocation.PLUGINS, JarLocation.LIB),
FACTORY),
TestParams.from("Jar in 'plugins'",
builder -> builder.copyJar(PROMETHEUS_JAR_PREFIX, JarLocation.OPT, JarLocation.PLUGINS),
builder -> {},
FACTORY),
TestParams.from("Jar in 'lib' and 'plugins'",
builder -> {
builder.copyJar(PROMETHEUS_JAR_PREFIX, JarLocation.OPT, JarLocation.LIB);
builder.copyJar(PROMETHEUS_JAR_PREFIX, JarLocation.OPT, JarLocation.PLUGINS);
builder.copyJar(PROMETHEUS_JAR_PREFIX, JarLocation.PLUGINS, JarLocation.LIB);
},
FACTORY)
);
Expand Down
1 change: 0 additions & 1 deletion flink-end-to-end-tests/test-scripts/common.sh
Original file line number Diff line number Diff line change
Expand Up @@ -596,7 +596,6 @@ function kill_random_taskmanager {

function setup_flink_slf4j_metric_reporter() {
INTERVAL="${1:-1 SECONDS}"
add_optional_plugin "metrics-slf4j"
set_config_key "metrics.reporter.slf4j.factory.class" "org.apache.flink.metrics.slf4j.Slf4jReporterFactory"
set_config_key "metrics.reporter.slf4j.interval" "${INTERVAL}"
}
Expand Down

0 comments on commit 3623967

Please sign in to comment.