本文档是 Apache Flink 的旧版本。建议访问 最新的稳定版本

指标

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 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, Histograms and Meters.

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.

public class MyMapper extends RichMapFunction<String, String> {
  private transient Counter counter;

  @Override
  public void open(Configuration config) {
    this.counter = getRuntimeContext()
      .getMetricGroup()
      .counter("myCounter");
  }

  @Override
  public String map(String value) throws Exception {
    this.counter.inc();
    return value;
  }
}
class MyMapper extends RichMapFunction[String,String] {
  @transient private var counter: Counter = _

  override def open(parameters: Configuration): Unit = {
    counter = getRuntimeContext()
      .getMetricGroup()
      .counter("myCounter")
  }

  override def map(value: String): String = {
    counter.inc()
    value
  }
}

Alternatively you can also use your own Counter implementation:

public class MyMapper extends RichMapFunction<String, String> {
  private transient Counter counter;

  @Override
  public void open(Configuration config) {
    this.counter = getRuntimeContext()
      .getMetricGroup()
      .counter("myCustomCounter", new CustomCounter());
  }

  @Override
  public String map(String value) throws Exception {
    this.counter.inc();
    return value;
  }
}
class MyMapper extends RichMapFunction[String,String] {
  @transient private var counter: Counter = _

  override def open(parameters: Configuration): Unit = {
    counter = getRuntimeContext()
      .getMetricGroup()
      .counter("myCustomCounter", new CustomCounter())
  }

  override def map(value: String): String = {
    counter.inc()
    value
  }
}

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.

public class MyMapper extends RichMapFunction<String, String> {
  private transient int valueToExpose = 0;

  @Override
  public void open(Configuration config) {
    getRuntimeContext()
      .getMetricGroup()
      .gauge("MyGauge", new Gauge<Integer>() {
        @Override
        public Integer getValue() {
          return valueToExpose;
        }
      });
  }

  @Override
  public String map(String value) throws Exception {
    valueToExpose++;
    return value;
  }
}
new class MyMapper extends RichMapFunction[String,String] {
  @transient private var valueToExpose = 0

  override def open(parameters: Configuration): Unit = {
    getRuntimeContext()
      .getMetricGroup()
      .gauge[Int, ScalaGauge[Int]]("MyGauge", ScalaGauge[Int]( () => valueToExpose ) )
  }

  override def map(value: String): String = {
    valueToExpose += 1
    value
  }
}

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.

public class MyMapper extends RichMapFunction<Long, Long> {
  private transient Histogram histogram;

  @Override
  public void open(Configuration config) {
    this.histogram = getRuntimeContext()
      .getMetricGroup()
      .histogram("myHistogram", new MyHistogram());
  }

  @Override
  public Long map(Long value) throws Exception {
    this.histogram.update(value);
    return value;
  }
}
class MyMapper extends RichMapFunction[Long,Long] {
  @transient private var histogram: Histogram = _

  override def open(parameters: Configuration): Unit = {
    histogram = getRuntimeContext()
      .getMetricGroup()
      .histogram("myHistogram", new MyHistogram())
  }

  override def map(value: Long): Long = {
    histogram.update(value)
    value
  }
}

Flink does not provide a default implementation for Histogram, but offers a Wrapper that allows usage of Codahale/DropWizard histograms. To use this wrapper add the following dependency in your pom.xml:

<dependency>
      <groupId>org.apache.flink</groupId>
      <artifactId>flink-metrics-dropwizard</artifactId>
      <version>1.11.6</version>
</dependency>

You can then register a Codahale/DropWizard histogram like this:

public class MyMapper extends RichMapFunction<Long, Long> {
  private transient Histogram histogram;

  @Override
  public void open(Configuration config) {
    com.codahale.metrics.Histogram dropwizardHistogram =
      new com.codahale.metrics.Histogram(new SlidingWindowReservoir(500));

    this.histogram = getRuntimeContext()
      .getMetricGroup()
      .histogram("myHistogram", new DropwizardHistogramWrapper(dropwizardHistogram));
  }

  @Override
  public Long map(Long value) throws Exception {
    this.histogram.update(value);
    return value;
  }
}
class MyMapper extends RichMapFunction[Long, Long] {
  @transient private var histogram: Histogram = _

  override def open(config: Configuration): Unit = {
    com.codahale.metrics.Histogram dropwizardHistogram =
      new com.codahale.metrics.Histogram(new SlidingWindowReservoir(500))

    histogram = getRuntimeContext()
      .getMetricGroup()
      .histogram("myHistogram", new DropwizardHistogramWrapper(dropwizardHistogram))
  }

  override def map(value: Long): Long = {
    histogram.update(value)
    value
  }
}

Meter

A Meter measures an average throughput. An occurrence of an event can be registered with the markEvent() method. Occurrence of multiple events at the same time can be registered with markEvent(long n) method. You can register a meter by calling meter(String name, Meter meter) on a MetricGroup.

public class MyMapper extends RichMapFunction<Long, Long> {
  private transient Meter meter;

  @Override
  public void open(Configuration config) {
    this.meter = getRuntimeContext()
      .getMetricGroup()
      .meter("myMeter", new MyMeter());
  }

  @Override
  public Long map(Long value) throws Exception {
    this.meter.markEvent();
    return value;
  }
}
class MyMapper extends RichMapFunction[Long,Long] {
  @transient private var meter: Meter = _

  override def open(config: Configuration): Unit = {
    meter = getRuntimeContext()
      .getMetricGroup()
      .meter("myMeter", new MyMeter())
  }

  override def map(value: Long): Long = {
    meter.markEvent()
    value
  }
}

Flink offers a Wrapper that allows usage of Codahale/DropWizard meters. To use this wrapper add the following dependency in your pom.xml:

<dependency>
      <groupId>org.apache.flink</groupId>
      <artifactId>flink-metrics-dropwizard</artifactId>
      <version>1.11.6</version>
</dependency>

You can then register a Codahale/DropWizard meter like this:

public class MyMapper extends RichMapFunction<Long, Long> {
  private transient Meter meter;

  @Override
  public void open(Configuration config) {
    com.codahale.metrics.Meter dropwizardMeter = new com.codahale.metrics.Meter();

    this.meter = getRuntimeContext()
      .getMetricGroup()
      .meter("myMeter", new DropwizardMeterWrapper(dropwizardMeter));
  }

  @Override
  public Long map(Long value) throws Exception {
    this.meter.markEvent();
    return value;
  }
}
class MyMapper extends RichMapFunction[Long,Long] {
  @transient private var meter: Meter = _

  override def open(config: Configuration): Unit = {
    com.codahale.metrics.Meter dropwizardMeter = new com.codahale.metrics.Meter()

    meter = getRuntimeContext()
      .getMetricGroup()
      .meter("myMeter", new DropwizardMeterWrapper(dropwizardMeter))
  }

  override def map(value: Long): Long = {
    meter.markEvent()
    value
  }
}

Scope

Every metric is assigned an identifier and a set of key-value pairs under which the metric will be reported.

The identifier is based on 3 components: a user-defined name when registering the metric, an optional user-defined scope and a system-provided scope. For example, if A.B is the system scope, C.D the user scope and E the name, then the identifier for the metric will be A.B.C.D.E.

You can configure which delimiter to use for the identifier (default: .) by setting the metrics.scope.delimiter key in conf/flink-conf.yaml.

User Scope

You can define a user scope by calling MetricGroup#addGroup(String name), MetricGroup#addGroup(int name) or MetricGroup#addGroup(String key, String value). These methods affect what MetricGroup#getMetricIdentifier and MetricGroup#getScopeComponents return.

counter = getRuntimeContext()
  .getMetricGroup()
  .addGroup("MyMetrics")
  .counter("myCounter");

counter = getRuntimeContext()
  .getMetricGroup()
  .addGroup("MyMetricsKey", "MyMetricsValue")
  .counter("myCounter");
counter = getRuntimeContext()
  .getMetricGroup()
  .addGroup("MyMetrics")
  .counter("myCounter")

counter = getRuntimeContext()
  .getMetricGroup()
  .addGroup("MyMetricsKey", "MyMetricsValue")
  .counter("myCounter")

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. “<task_id>”) which will be replaced at runtime.

  • metrics.scope.jm
    • Default: <host>.jobmanager
    • Applied to all metrics that were scoped to a job manager.
  • metrics.scope.jm.job
    • Default: <host>.jobmanager.<job_name>
    • Applied to all metrics that were scoped to a job manager and job.
  • metrics.scope.tm
    • Default: <host>.taskmanager.<tm_id>
    • Applied to all metrics that were scoped to a task manager.
  • metrics.scope.tm.job
    • Default: <host>.taskmanager.<tm_id>.<job_name>
    • Applied to all metrics that were scoped to a task manager and job.
  • metrics.scope.task
    • Default: <host>.taskmanager.<tm_id>.<job_name>.<task_name>.<subtask_index>
    • Applied to all metrics that were scoped to a task.
  • metrics.scope.operator
    • Default: <host>.taskmanager.<tm_id>.<job_name>.<operator_name>.<subtask_index>
    • 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.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 <job_id>) or by assigning unique names to jobs and operators.

List of all Variables

  • 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_id>,<operator_name>, <subtask_index>

Important: For the Batch API, <operator_id> is always equal to <task_id>.

User Variables

You can define a user variable by calling MetricGroup#addGroup(String key, String value). This method affects what MetricGroup#getMetricIdentifier, MetricGroup#getScopeComponents and MetricGroup#getAllVariables() returns.

Important: User variables cannot be used in scope formats.

counter = getRuntimeContext()
  .getMetricGroup()
  .addGroup("MyMetricsKey", "MyMetricsValue")
  .counter("myCounter");
counter = getRuntimeContext()
  .getMetricGroup()
  .addGroup("MyMetricsKey", "MyMetricsValue")
  .counter("myCounter")

Reporter

Metrics can be exposed to an external system by configuring one or several reporters in conf/flink-conf.yaml. These reporters will be instantiated on each job and task manager when they are started.

  • metrics.reporter.<name>.<config>: Generic setting <config> for the reporter named <name>.
  • metrics.reporter.<name>.class: The reporter class to use for the reporter named <name>.
  • metrics.reporter.<name>.factory.class: The reporter factory class to use for the reporter named <name>.
  • metrics.reporter.<name>.interval: The reporter interval to use for the reporter named <name>.
  • metrics.reporter.<name>.scope.delimiter: The delimiter to use for the identifier (default value use metrics.scope.delimiter) for the reporter named <name>.
  • metrics.reporter.<name>.scope.variables.excludes: (optional) A semi-colon (;) separate list of variables that should be ignored by tag-based reporters (e.g., Prometheus, InfluxDB).
  • metrics.reporters: (optional) A comma-separated include list of reporter names. By default all configured reporters will be used.

All reporters must at least have either the class or factory.class property. Which property may/should be used depends on the reporter implementation. See the individual reporter configuration sections for more information. Some reporters (referred to as Scheduled) allow specifying a reporting interval. Below more settings specific to each reporter will be listed.

Example reporter configuration that specifies multiple reporters:

metrics.reporters: my_jmx_reporter,my_other_reporter

metrics.reporter.my_jmx_reporter.factory.class: org.apache.flink.metrics.jmx.JMXReporterFactory
metrics.reporter.my_jmx_reporter.port: 9020-9040
metrics.reporter.my_jmx_reporter.scope.variables.excludes:job_id;task_attempt_num

metrics.reporter.my_other_reporter.class: org.apache.flink.metrics.graphite.GraphiteReporter
metrics.reporter.my_other_reporter.host: 192.168.1.1
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. Otherwise the jar must be placed 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.

JMX (org.apache.flink.metrics.jmx.JMXReporter)

You don’t have to include an additional dependency since the JMX reporter is available by default but not activated.

Parameters:

  • port - (optional) the port on which JMX listens for connections. 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. When a range is specified the actual port is shown in the relevant job or task manager log. If this setting is set Flink will start an extra JMX connector for the given port/range. Metrics are always available on the default local JMX interface.

Example configuration:

metrics.reporter.jmx.factory.class: org.apache.flink.metrics.jmx.JMXReporterFactory
metrics.reporter.jmx.port: 8789

Metrics exposed through JMX are identified by a domain and a list of key-properties, which together form the object name.

The domain always begins with org.apache.flink followed by a generalized metric identifier. In contrast to the usual identifier it is not affected by scope-formats, does not contain any variables and is constant across jobs. An example for such a domain would be org.apache.flink.job.task.numBytesOut.

The key-property list contains the values for all variables, regardless of configured scope formats, that are associated with a given metric. An example for such a list would be host=localhost,job_name=MyJob,task_name=MyTask.

The domain thus identifies a metric class, while the key-property list identifies one (or multiple) instances of that metric.

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

Parameters:

  • host - the Graphite server host
  • port - the Graphite server port
  • protocol - protocol to use (TCP/UDP)

Example configuration:

metrics.reporter.grph.factory.class: org.apache.flink.metrics.graphite.GraphiteReporterFactory
metrics.reporter.grph.host: localhost
metrics.reporter.grph.port: 2003
metrics.reporter.grph.protocol: TCP
metrics.reporter.grph.interval: 60 SECONDS

InfluxDB (org.apache.flink.metrics.influxdb.InfluxdbReporter)

In order to use this reporter you must copy /opt/flink-metrics-influxdb-1.11.6.jar into the plugins/influxdb folder of your Flink distribution.

Parameters:

Key Default Type Description
connectTimeout
10000 Integer (optional) the InfluxDB connect timeout for metrics
consistency
ONE

Enum

Possible values: [ALL, ANY, ONE, QUORUM]
(optional) the InfluxDB consistency level for metrics
db
(none) String the InfluxDB database to store metrics
host
(none) String the InfluxDB server host
password
(none) String (optional) InfluxDB username's password used for authentication
port
8086 Integer the InfluxDB server port
retentionPolicy
(none) String (optional) the InfluxDB retention policy for metrics
username
(none) String (optional) InfluxDB username used for authentication
writeTimeout
10000 Integer (optional) the InfluxDB write timeout for metrics

Example configuration:

metrics.reporter.influxdb.factory.class: org.apache.flink.metrics.influxdb.InfluxdbReporterFactory
metrics.reporter.influxdb.host: localhost
metrics.reporter.influxdb.port: 8086
metrics.reporter.influxdb.db: flink
metrics.reporter.influxdb.username: flink-metrics
metrics.reporter.influxdb.password: qwerty
metrics.reporter.influxdb.retentionPolicy: one_hour
metrics.reporter.influxdb.consistency: ANY
metrics.reporter.influxdb.connectTimeout: 60000
metrics.reporter.influxdb.writeTimeout: 60000
metrics.reporter.influxdb.interval: 60 SECONDS

The reporter would send metrics using http protocol to the InfluxDB server with the specified retention policy (or the default policy specified on the server). All Flink metrics variables (see List of all Variables) are exported as InfluxDB tags.

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

Parameters:

  • port - (optional) the port the Prometheus exporter listens on, defaults to 9249. 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.
  • filterLabelValueCharacters - (optional) Specifies whether to filter label value characters. If enabled, all characters not matching [a-zA-Z0-9:_] will be removed, otherwise no characters will be removed. Before disabling this option please ensure that your label values meet the Prometheus requirements.

Example configuration:

metrics.reporter.prom.class: org.apache.flink.metrics.prometheus.PrometheusReporter

Flink metric types are mapped to Prometheus metric types as follows:

Flink Prometheus Note
Counter Gauge Prometheus counters cannot be decremented.
Gauge Gauge Only numbers and booleans are supported.
Histogram Summary Quantiles .5, .75, .95, .98, .99 and .999
Meter Gauge The gauge exports the meter’s rate.

All Flink metrics variables (see List of all Variables) are exported to Prometheus as labels.

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

Parameters:

Key Default Type Description
deleteOnShutdown
true Boolean Specifies whether to delete metrics from the PushGateway on shutdown.
filterLabelValueCharacters
true Boolean Specifies whether to filter label value characters. If enabled, all characters not matching [a-zA-Z0-9:_] will be removed, otherwise no characters will be removed. Before disabling this option please ensure that your label values meet the Prometheus requirements.
groupingKey
(none) String Specifies the grouping key which is the group and global labels of all metrics. The label name and value are separated by '=', and labels are separated by ';', e.g., k1=v1;k2=v2. Please ensure that your grouping key meets the Prometheus requirements.
host
(none) String The PushGateway server host.
jobName
(none) String The job name under which metrics will be pushed
port
-1 Integer The PushGateway server port.
randomJobNameSuffix
true Boolean Specifies whether a random suffix should be appended to the job name.

Example configuration:

metrics.reporter.promgateway.class: org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporter
metrics.reporter.promgateway.host: localhost
metrics.reporter.promgateway.port: 9091
metrics.reporter.promgateway.jobName: myJob
metrics.reporter.promgateway.randomJobNameSuffix: true
metrics.reporter.promgateway.deleteOnShutdown: false
metrics.reporter.promgateway.groupingKey: k1=v1;k2=v2
metrics.reporter.promgateway.interval: 60 SECONDS

The PrometheusPushGatewayReporter pushes metrics to a Pushgateway, which can be scraped by Prometheus.

Please see the Prometheus documentation for use-cases.

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

Parameters:

  • host - the StatsD server host
  • port - the StatsD server port

Example configuration:

metrics.reporter.stsd.factory.class: org.apache.flink.metrics.statsd.StatsDReporterFactory
metrics.reporter.stsd.host: localhost
metrics.reporter.stsd.port: 8125
metrics.reporter.stsd.interval: 60 SECONDS

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

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.

Parameters:

  • apikey - the Datadog API key
  • tags - (optional) the global tags that will be applied to metrics when sending to Datadog. Tags should be separated by comma only
  • proxyHost - (optional) The proxy host to use when sending to Datadog.
  • proxyPort - (optional) The proxy port to use when sending to Datadog, defaults to 8080.
  • dataCenter - (optional) The data center (EU/US) to connect to, defaults to US.
  • maxMetricsPerRequest - (optional) The maximum number of metrics to include in each request, defaults to 2000.

Example configuration:

metrics.reporter.dghttp.factory.class: org.apache.flink.metrics.datadog.DatadogHttpReporterFactory
metrics.reporter.dghttp.apikey: xxx
metrics.reporter.dghttp.tags: myflinkapp,prod
metrics.reporter.dghttp.proxyHost: my.web.proxy.com
metrics.reporter.dghttp.proxyPort: 8080
metrics.reporter.dghttp.dataCenter: US
metrics.reporter.dghttp.maxMetricsPerRequest: 2000
metrics.reporter.dghttp.interval: 60 SECONDS

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

Example configuration:

metrics.reporter.slf4j.factory.class: org.apache.flink.metrics.slf4j.Slf4jReporterFactory
metrics.reporter.slf4j.interval: 60 SECONDS

System metrics

By default Flink gathers several metrics that provide deep insights on the current state. This section is a reference of all these metrics.

The tables below generally feature 5 columns:

  • The “Scope” column describes which scope format is used to generate the system scope. For example, if the cell contains “Operator” then the scope format for “metrics.scope.operator” is used. If the cell contains multiple values, separated by a slash, then the metrics are reported multiple times for different entities, like for both job- and taskmanagers.

  • The (optional)”Infix” column describes which infix is appended to the system scope.

  • The “Metrics” column lists the names of all metrics that are registered for the given scope and infix.

  • The “Description” column provides information as to what a given metric is measuring.

  • The “Type” column describes which metric type is used for the measurement.

Note that all dots in the infix/metric name columns are still subject to the “metrics.delimiter” setting.

Thus, in order to infer the metric identifier:

  1. Take the scope-format based on the “Scope” column
  2. Append the value in the “Infix” column if present, and account for the “metrics.delimiter” setting
  3. Append metric name.

CPU

Scope Infix Metrics Description Type
Job-/TaskManager Status.JVM.CPU Load The recent CPU usage of the JVM. Gauge
Time The CPU time used by the JVM. Gauge

Memory

Scope Infix Metrics Description Type
Job-/TaskManager Status.JVM.Memory Heap.Used The amount of heap memory currently used (in bytes). Gauge
Heap.Committed The amount of heap memory guaranteed to be available to the JVM (in bytes). Gauge
Heap.Max The maximum amount of heap memory that can be used for memory management (in bytes). Gauge
NonHeap.Used The amount of non-heap memory currently used (in bytes). Gauge
NonHeap.Committed The amount of non-heap memory guaranteed to be available to the JVM (in bytes). Gauge
NonHeap.Max The maximum amount of non-heap memory that can be used for memory management (in bytes). Gauge
Direct.Count The number of buffers in the direct buffer pool. Gauge
Direct.MemoryUsed The amount of memory used by the JVM for the direct buffer pool (in bytes). Gauge
Direct.TotalCapacity The total capacity of all buffers in the direct buffer pool (in bytes). Gauge
Mapped.Count The number of buffers in the mapped buffer pool. Gauge
Mapped.MemoryUsed The amount of memory used by the JVM for the mapped buffer pool (in bytes). Gauge
Mapped.TotalCapacity The number of buffers in the mapped buffer pool (in bytes). Gauge

Threads

Scope Infix Metrics Description Type
Job-/TaskManager Status.JVM.Threads Count The total number of live threads. Gauge

GarbageCollection

Scope Infix Metrics Description Type
Job-/TaskManager Status.JVM.GarbageCollector <GarbageCollector>.Count The total number of collections that have occurred. Gauge
<GarbageCollector>.Time The total time spent performing garbage collection. Gauge

ClassLoader

Scope Infix Metrics Description Type
Job-/TaskManager Status.JVM.ClassLoader ClassesLoaded The total number of classes loaded since the start of the JVM. Gauge
ClassesUnloaded The total number of classes unloaded since the start of the JVM. Gauge

Network (Deprecated: use Default shuffle service metrics)

Scope Infix Metrics Description Type
TaskManager Status.Network AvailableMemorySegments The number of unused memory segments. Gauge
TotalMemorySegments The number of allocated memory segments. Gauge
Task buffers inputQueueLength The number of queued input buffers. (ignores LocalInputChannels which are using blocking subpartitions) Gauge
outputQueueLength The number of queued output buffers. Gauge
inPoolUsage An estimate of the input buffers usage. (ignores LocalInputChannels) Gauge
inputFloatingBuffersUsage An estimate of the floating input buffers usage. (ignores LocalInputChannels) Gauge
inputExclusiveBuffersUsage An estimate of the exclusive input buffers usage. (ignores LocalInputChannels) Gauge
outPoolUsage An estimate of the output buffers usage. Gauge
Network.<Input|Output>.<gate|partition>
(only available if taskmanager.net.detailed-metrics config option is set)
totalQueueLen Total number of queued buffers in all input/output channels. Gauge
minQueueLen Minimum number of queued buffers in all input/output channels. Gauge
maxQueueLen Maximum number of queued buffers in all input/output channels. Gauge
avgQueueLen Average number of queued buffers in all input/output channels. Gauge

Default shuffle service

Metrics related to data exchange between task executors using netty network communication.

Scope Infix Metrics Description Type
TaskManager Status.Shuffle.Netty AvailableMemorySegments The number of unused memory segments. Gauge
TotalMemorySegments The number of allocated memory segments. Gauge
Task Shuffle.Netty.Input.Buffers inputQueueLength The number of queued input buffers. Gauge
inPoolUsage An estimate of the input buffers usage. Gauge
Shuffle.Netty.Output.Buffers outputQueueLength The number of queued output buffers. Gauge
outPoolUsage An estimate of the output buffers usage. Gauge
Shuffle.Netty.<Input|Output>.<gate|partition>
(only available if taskmanager.net.detailed-metrics config option is set)
totalQueueLen Total number of queued buffers in all input/output channels. Gauge
minQueueLen Minimum number of queued buffers in all input/output channels. Gauge
maxQueueLen Maximum number of queued buffers in all input/output channels. Gauge
avgQueueLen Average number of queued buffers in all input/output channels. Gauge
Task Shuffle.Netty.Input numBytesInLocal The total number of bytes this task has read from a local source. Counter
numBytesInLocalPerSecond The number of bytes this task reads from a local source per second. Meter
numBytesInRemote The total number of bytes this task has read from a remote source. Counter
numBytesInRemotePerSecond The number of bytes this task reads from a remote source per second. Meter
numBuffersInLocal The total number of network buffers this task has read from a local source. Counter
numBuffersInLocalPerSecond The number of network buffers this task reads from a local source per second. Meter
numBuffersInRemote The total number of network buffers this task has read from a remote source. Counter
numBuffersInRemotePerSecond The number of network buffers this task reads from a remote source per second. Meter

Cluster

Scope Metrics Description Type
JobManager numRegisteredTaskManagers The number of registered taskmanagers. Gauge
numRunningJobs The number of running jobs. Gauge
taskSlotsAvailable The number of available task slots. Gauge
taskSlotsTotal The total number of task slots. Gauge

Availability

Scope Metrics Description Type
Job (only available on JobManager) restartingTime The time it took to restart the job, or how long the current restart has been in progress (in milliseconds). Gauge
uptime The time that the job has been running without interruption.

Returns -1 for completed jobs (in milliseconds).

Gauge
downtime For jobs currently in a failing/recovering situation, the time elapsed during this outage.

Returns 0 for running jobs and -1 for completed jobs (in milliseconds).

Gauge
fullRestarts Attention: deprecated, use numRestarts. Gauge
numRestarts The total number of restarts since this job was submitted, including full restarts and fine-grained restarts. Gauge

Checkpointing

Scope Metrics Description Type
Job (only available on JobManager) lastCheckpointDuration The time it took to complete the last checkpoint (in milliseconds). Gauge
lastCheckpointSize The total size of the last checkpoint (in bytes). Gauge
lastCheckpointExternalPath The path where the last external checkpoint was stored. Gauge
lastCheckpointRestoreTimestamp Timestamp when the last checkpoint was restored at the coordinator (in milliseconds). Gauge
numberOfInProgressCheckpoints The number of in progress checkpoints. Gauge
numberOfCompletedCheckpoints The number of successfully completed checkpoints. Gauge
numberOfFailedCheckpoints The number of failed checkpoints. Gauge
totalNumberOfCheckpoints The number of total checkpoints (in progress, completed, failed). Gauge
Task checkpointAlignmentTime The time in nanoseconds that the last barrier alignment took to complete, or how long the current alignment has taken so far (in nanoseconds). Gauge
checkpointStartDelayNanos The time in nanoseconds that elapsed between the creation of the last checkpoint and the time when the checkpointing process has started by this Task. This delay shows how long it takes for the first checkpoint barrier to reach the task. A high value indicates back-pressure. If only a specific task has a long start delay, the most likely reason is data skew. Gauge

RocksDB

Certain RocksDB native metrics are available but disabled by default, you can find full documentation here

IO

Scope Metrics Description Type
Job (only available on TaskManager) [<source_id>.[<source_subtask_index>.]]<operator_id>.<operator_subtask_index>.latency The latency distributions from a given source (subtask) to an operator subtask (in milliseconds), depending on the latency granularity. Histogram
Task numBytesInLocal Attention: deprecated, use Default shuffle service metrics. Counter
numBytesInLocalPerSecond Attention: deprecated, use Default shuffle service metrics. Meter
numBytesInRemote Attention: deprecated, use Default shuffle service metrics. Counter
numBytesInRemotePerSecond Attention: deprecated, use Default shuffle service metrics. Meter
numBuffersInLocal Attention: deprecated, use Default shuffle service metrics. Counter
numBuffersInLocalPerSecond Attention: deprecated, use Default shuffle service metrics. Meter
numBuffersInRemote Attention: deprecated, use Default shuffle service metrics. Counter
numBuffersInRemotePerSecond Attention: deprecated, use Default shuffle service metrics. Meter
numBytesOut The total number of bytes this task has emitted. Counter
numBytesOutPerSecond The number of bytes this task emits per second. Meter
numBuffersOut The total number of network buffers this task has emitted. Counter
numBuffersOutPerSecond The number of network buffers this task emits per second. Meter
isBackPressured Whether the task is back-pressured. Gauge
idleTimeMsPerSecond The time (in milliseconds) this task is idle (either has no data to process or it is back pressured) per second. Meter
Task/Operator numRecordsIn The total number of records this operator/task has received. Counter
numRecordsInPerSecond The number of records this operator/task receives per second. Meter
numRecordsOut The total number of records this operator/task has emitted. Counter
numRecordsOutPerSecond The number of records this operator/task sends per second. Meter
numLateRecordsDropped The number of records this operator/task has dropped due to arriving late. Counter
currentInputWatermark The last watermark this operator/tasks has received (in milliseconds).

Note: For operators/tasks with 2 inputs this is the minimum of the last received watermarks.

Gauge
Operator currentInputNWatermark The last watermark this operator has received in its N'th input (in milliseconds), with index N starting from 1. For example currentInput1Watermark, currentInput2Watermark, ...

Note: Only for operators with 2 or more inputs.

Gauge
currentOutputWatermark The last watermark this operator has emitted (in milliseconds). Gauge
numSplitsProcessed The total number of InputSplits this data source has processed (if the operator is a data source). Gauge

Connectors

Kafka Connectors

Scope Metrics User Variables Description Type
Operator commitsSucceeded n/a The total number of successful offset commits to Kafka, if offset committing is turned on and checkpointing is enabled. Counter
Operator commitsFailed n/a The total number of offset commit failures to Kafka, if offset committing is turned on and checkpointing is enabled. Note that committing offsets back to Kafka is only a means to expose consumer progress, so a commit failure does not affect the integrity of Flink's checkpointed partition offsets. Counter
Operator committedOffsets topic, partition The last successfully committed offsets to Kafka, for each partition. A particular partition's metric can be specified by topic name and partition id. Gauge
Operator currentOffsets topic, partition The consumer's current read offset, for each partition. A particular partition's metric can be specified by topic name and partition id. Gauge

Kinesis Connectors

Scope Metrics User Variables Description Type
Operator millisBehindLatest stream, shardId The number of milliseconds the consumer is behind the head of the stream, indicating how far behind current time the consumer is, for each Kinesis shard. A particular shard's metric can be specified by stream name and shard id. A value of 0 indicates record processing is caught up, and there are no new records to process at this moment. A value of -1 indicates that there is no reported value for the metric, yet. Gauge
Operator sleepTimeMillis stream, shardId The number of milliseconds the consumer spends sleeping before fetching records from Kinesis. A particular shard's metric can be specified by stream name and shard id. Gauge
Operator maxNumberOfRecordsPerFetch stream, shardId The maximum number of records requested by the consumer in a single getRecords call to Kinesis. If ConsumerConfigConstants.SHARD_USE_ADAPTIVE_READS is set to true, this value is adaptively calculated to maximize the 2 Mbps read limits from Kinesis. Gauge
Operator numberOfAggregatedRecordsPerFetch stream, shardId The number of aggregated Kinesis records fetched by the consumer in a single getRecords call to Kinesis. Gauge
Operator numberOfDeggregatedRecordsPerFetch stream, shardId The number of deaggregated Kinesis records fetched by the consumer in a single getRecords call to Kinesis. Gauge
Operator averageRecordSizeBytes stream, shardId The average size of a Kinesis record in bytes, fetched by the consumer in a single getRecords call. Gauge
Operator runLoopTimeNanos stream, shardId The actual time taken, in nanoseconds, by the consumer in the run loop. Gauge
Operator loopFrequencyHz stream, shardId The number of calls to getRecords in one second. Gauge
Operator bytesRequestedPerFetch stream, shardId The bytes requested (2 Mbps / loopFrequencyHz) in a single call to getRecords. Gauge

System resources

System resources reporting is disabled by default. When metrics.system-resource is enabled additional metrics listed below will be available on Job- and TaskManager. System resources metrics are updated periodically and they present average values for a configured interval (metrics.system-resource-probing-interval).

System resources reporting requires an optional dependency to be present on the classpath (for example placed in Flink’s lib directory):

  • com.github.oshi:oshi-core:3.4.0 (licensed under EPL 1.0 license)

Including it’s transitive dependencies:

  • net.java.dev.jna:jna-platform:jar:4.2.2
  • net.java.dev.jna:jna:jar:4.2.2

Failures in this regard will be reported as warning messages like NoClassDefFoundError logged by SystemResourcesMetricsInitializer during the startup.

System CPU

Scope Infix Metrics Description
Job-/TaskManager System.CPU Usage Overall % of CPU usage on the machine.
Idle % of CPU Idle usage on the machine.
Sys % of System CPU usage on the machine.
User % of User CPU usage on the machine.
IOWait % of IOWait CPU usage on the machine.
Irq % of Irq CPU usage on the machine.
SoftIrq % of SoftIrq CPU usage on the machine.
Nice % of Nice Idle usage on the machine.
Load1min Average CPU load over 1 minute
Load5min Average CPU load over 5 minute
Load15min Average CPU load over 15 minute
UsageCPU* % of CPU usage per each processor

System memory

Scope Infix Metrics Description
Job-/TaskManager System.Memory Available Available memory in bytes
Total Total memory in bytes
System.Swap Used Used swap bytes
Total Total swap in bytes

System network

Scope Infix Metrics Description
Job-/TaskManager System.Network.INTERFACE_NAME ReceiveRate Average receive rate in bytes per second
SendRate Average send rate in bytes per second

Latency tracking

Flink allows to track the latency of records travelling through the system. This feature is disabled by default. To enable the latency tracking you must set the latencyTrackingInterval to a positive number in either the Flink configuration or ExecutionConfig.

At the latencyTrackingInterval, the sources will periodically emit a special record, called a LatencyMarker. The marker contains a timestamp from the time when the record has been emitted at the sources. Latency markers can not overtake regular user records, thus if records are queuing up in front of an operator, it will add to the latency tracked by the marker.

Note that the latency markers are not accounting for the time user records spend in operators as they are bypassing them. In particular the markers are not accounting for the time records spend for example in window buffers. Only if operators are not able to accept new records, thus they are queuing up, the latency measured using the markers will reflect that.

The LatencyMarkers are used to derive a distribution of the latency between the sources of the topology and each downstream operator. These distributions are reported as histogram metrics. The granularity of these distributions can be controlled in the Flink configuration. For the highest granularity subtask Flink will derive the latency distribution between every source subtask and every downstream subtask, which results in quadratic (in the terms of the parallelism) number of histograms.

Currently, Flink assumes that the clocks of all machines in the cluster are in sync. We recommend setting up an automated clock synchronisation service (like NTP) to avoid false latency results.

Warning Enabling latency metrics can significantly impact the performance of the cluster (in particular for subtask granularity). It is highly recommended to only use them for debugging purposes.

REST API integration

Metrics can be queried through the Monitoring REST API.

Below is a list of available endpoints, with a sample JSON response. All endpoints are of the sample form http://hostname:8081/jobmanager/metrics, below we list only the path part of the URLs.

Values in angle brackets are variables, for example http://hostname:8081/jobs/<jobid>/metrics will have to be requested for example as http://hostname:8081/jobs/7684be6004e4e955c2a558a9bc463f65/metrics.

Request metrics for a specific entity:

  • /jobmanager/metrics
  • /taskmanagers/<taskmanagerid>/metrics
  • /jobs/<jobid>/metrics
  • /jobs/<jobid>/vertices/<vertexid>/subtasks/<subtaskindex>

Request metrics aggregated across all entities of the respective type:

  • /taskmanagers/metrics
  • /jobs/metrics
  • /jobs/<jobid>/vertices/<vertexid>/subtasks/metrics

Request metrics aggregated over a subset of all entities of the respective type:

  • /taskmanagers/metrics?taskmanagers=A,B,C
  • /jobs/metrics?jobs=D,E,F
  • /jobs/<jobid>/vertices/<vertexid>/subtasks/metrics?subtask=1,2,3

Request a list of available metrics:

GET /jobmanager/metrics

[
  {
    "id": "metric1"
  },
  {
    "id": "metric2"
  }
]

Request the values for specific (unaggregated) metrics:

GET taskmanagers/ABCDE/metrics?get=metric1,metric2

[
  {
    "id": "metric1",
    "value": "34"
  },
  {
    "id": "metric2",
    "value": "2"
  }
]

Request aggregated values for specific metrics:

GET /taskmanagers/metrics?get=metric1,metric2

[
  {
    "id": "metric1",
    "min": 1,
    "max": 34,
    "avg": 15,
    "sum": 45
  },
  {
    "id": "metric2",
    "min": 2,
    "max": 14,
    "avg": 7,
    "sum": 16
  }
]

Request specific aggregated values for specific metrics:

GET /taskmanagers/metrics?get=metric1,metric2&agg=min,max

[
  {
    "id": "metric1",
    "min": 1,
    "max": 34,
  },
  {
    "id": "metric2",
    "min": 2,
    "max": 14,
  }
]

Dashboard integration

Metrics that were gathered for each task or operator can also be visualized in the Dashboard. On the main page for a job, select the Metrics tab. After selecting one of the tasks in the top graph you can select metrics to display using the Add Metric drop-down menu.

  • Task metrics are listed as <subtask_index>.<metric_name>.
  • Operator metrics are listed as <subtask_index>.<operator_name>.<metric_name>.

Each metric will be visualized as a separate graph, with the x-axis representing time and the y-axis the measured value. All graphs are automatically updated every 10 seconds, and continue to do so when navigating to another page.

There is no limit as to the number of visualized metrics; however only numeric metrics can be visualized.

Back to top