[ https://issues.apache.org/jira/browse/FLINK-4389?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15426764#comment-15426764 ]
ASF GitHub Bot commented on FLINK-4389: --------------------------------------- Github user tillrohrmann commented on a diff in the pull request: https://github.com/apache/flink/pull/2363#discussion_r75342927 --- 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 --- End diff -- I think we should create proper message types instead of using a `Tuple3` here. > Expose metrics to Webfrontend > ----------------------------- > > Key: FLINK-4389 > URL: https://issues.apache.org/jira/browse/FLINK-4389 > Project: Flink > Issue Type: Sub-task > Components: Metrics, Webfrontend > Affects Versions: 1.1.0 > Reporter: Chesnay Schepler > Assignee: Chesnay Schepler > Fix For: pre-apache > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-7%3A+Expose+metrics+to+WebInterface -- This message was sent by Atlassian JIRA (v6.3.4#6332)