Github user tillrohrmann commented on a diff in the pull request: https://github.com/apache/flink/pull/2363#discussion_r75450989 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricQueryService.java --- @@ -0,0 +1,180 @@ +/* + * 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. + */ +package org.apache.flink.runtime.metrics; + +import akka.actor.ActorRef; +import akka.actor.ActorSystem; +import akka.actor.Props; +import akka.actor.UntypedActor; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.metrics.CharacterFilter; +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.Metric; +import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * The MetricQueryService creates a key-value representation of all metrics currently registered with Flink when queried. + * + * It is realized as an actor and receives the following messages: + * - {@code Tuple3<String, Metric, AbstractMetricGroup> => Notification of added Metric} + * - {@code Metric => Notification of removed Metric} + * - {@code ActorRef => Query for metric dump} + */ +public class MetricQueryService extends UntypedActor { + private static final Logger LOG = LoggerFactory.getLogger(MetricQueryService.class); + + public static final byte CATEGORY_COUNTER = 0; + public static final byte CATEGORY_GAUGE = 1; + public static final byte CATEGORY_HISTOGRAM = 2; + + private final Map<Gauge<?>, String> gauges = new HashMap<>(); + private final Map<Counter, String> counters = new HashMap<>(); + private final Map<Histogram, String> histograms = new HashMap<>(); + + private final List<Metric> toRemove = new ArrayList<>(); + + @Override + public void onReceive(Object message) throws Exception { + try { + if (message instanceof Tuple3) { // add metric + Tuple3<String, Metric, AbstractMetricGroup> tuple = (Tuple3<String, Metric, AbstractMetricGroup>) message; + + String metricName = tuple.f0; + Metric metric = tuple.f1; + AbstractMetricGroup group = tuple.f2; + + String name = group.getQueryServiceMetricIdentifier(metricName, new CharacterFilter() { + @Override + public String filterCharacters(String input) { + return input.replaceAll("[ :.,]", "_"); + } + }); + + if (metric instanceof Counter) { + counters.put((Counter) metric, name); + } else if (metric instanceof Gauge) { + gauges.put((Gauge<?>) metric, name); + } else if (metric instanceof Histogram) { + histograms.put((Histogram) metric, name); + } + } else if (message instanceof Metric) { // remove metric + toRemove.add((Metric) message); + } else if (message instanceof ActorRef) { // create dump + Object[] dump = createDump(); + ((ActorRef) message).tell(dump, getSelf()); + removeMetrics(); + } else { + LOG.warn("MetricQueryServiceActor received an invalid message. " + message.toString()); + } + } catch (Exception e) { + LOG.warn("An exception occurred while processing a message.", e); + } + } + + private Object[] createDump() { + Object[] metrics = new Object[(this.counters.size() + this.gauges.size()) * 3 + this.histograms.size() * 13]; + + int index = 0; + + // dump counters + for (Map.Entry<Counter, String> counter : this.counters.entrySet()) { + metrics[index++] = CATEGORY_COUNTER; + metrics[index++] = counter.getValue(); + metrics[index++] = counter.getKey().getCount(); + } + // dump gauges + for (Map.Entry<Gauge<?>, String> gauge : this.gauges.entrySet()) { + metrics[index++] = CATEGORY_GAUGE; + metrics[index++] = gauge.getValue(); + metrics[index++] = gauge.getKey().getValue().toString(); + } + // dump histograms + for (Map.Entry<Histogram, String> histogram : this.histograms.entrySet()) { + HistogramStatistics stat = histogram.getKey().getStatistics(); + + metrics[index++] = CATEGORY_HISTOGRAM; + metrics[index++] = histogram.getValue(); + metrics[index++] = stat.getMin(); + metrics[index++] = stat.getMax(); + metrics[index++] = stat.getMean(); + metrics[index++] = stat.getQuantile(0.5); + metrics[index++] = stat.getStdDev(); + metrics[index++] = stat.getQuantile(0.75); + metrics[index++] = stat.getQuantile(0.9); + metrics[index++] = stat.getQuantile(0.95); + metrics[index++] = stat.getQuantile(0.98); + metrics[index++] = stat.getQuantile(0.99); + metrics[index++] = stat.getQuantile(0.999); + } + return metrics; + } + + private void removeMetrics() { + for (Metric metric : toRemove) { + if (metric instanceof Counter) { + this.counters.remove(metric); + } else if (metric instanceof Gauge) { + this.gauges.remove(metric); + } else if (metric instanceof Histogram) { + this.histograms.remove(metric); + } + } + } + + /** + * Starts the MetricQueryService actor in the given actor system. + * + * @param actorSystem The actor system running the MetricQueryService + * @return actor reference to the MetricQueryService + */ + public static ActorRef startMetricQueryService(ActorSystem actorSystem) { + return actorSystem.actorOf(Props.create(MetricQueryService.class)); --- End diff -- `actorSystem.actorOf(Props.create(MetricQueryService.class), "foobar")` creates an actor with the name "foobar". If your actor system is the Flink actor system, then it should be reachable via `akka.tcp://fl...@xxx.yyy.zzz:port/foobar` if I'm not mistaken. But you would need an actorsystem reference in the web server to do the actor resolution.
--- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---