[ 
https://issues.apache.org/jira/browse/BEAM-6172?focusedWorklogId=174036&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-174036
 ]

ASF GitHub Bot logged work on BEAM-6172:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 11/Dec/18 11:37
            Start Date: 11/Dec/18 11:37
    Worklog Time Spent: 10m 
      Work Description: mxm closed pull request #7207: [BEAM-6172] Adjust Flink 
metric names / Add metric reporting tests
URL: https://github.com/apache/beam/pull/7207
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainer.java
 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainer.java
index 73fe62e219fc..f4857d18a587 100644
--- 
a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainer.java
+++ 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainer.java
@@ -19,12 +19,14 @@
 
 import static 
org.apache.beam.runners.core.metrics.MetricsContainerStepMap.asAttemptedOnlyMetricResults;
 
+import com.google.common.annotations.VisibleForTesting;
 import java.util.HashMap;
 import java.util.Map;
 import org.apache.beam.runners.core.metrics.MetricsContainerImpl;
 import org.apache.beam.runners.core.metrics.MetricsContainerStepMap;
 import org.apache.beam.sdk.metrics.DistributionResult;
 import org.apache.beam.sdk.metrics.GaugeResult;
+import org.apache.beam.sdk.metrics.MetricName;
 import org.apache.beam.sdk.metrics.MetricQueryResults;
 import org.apache.beam.sdk.metrics.MetricResult;
 import org.apache.beam.sdk.metrics.MetricResults;
@@ -32,6 +34,8 @@
 import org.apache.beam.sdk.metrics.MetricsFilter;
 import org.apache.flink.api.common.accumulators.Accumulator;
 import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.MetricOptions;
 import org.apache.flink.metrics.Counter;
 import org.apache.flink.metrics.Gauge;
 import org.slf4j.Logger;
@@ -47,10 +51,8 @@
 
   private static final Logger LOG = 
LoggerFactory.getLogger(FlinkMetricContainer.class);
 
-  private static final String METRIC_KEY_SEPARATOR = "__";
-  private static final String COUNTER_PREFIX = "__counter";
-  private static final String DISTRIBUTION_PREFIX = "__distribution";
-  private static final String GAUGE_PREFIX = "__gauge";
+  private static final String METRIC_KEY_SEPARATOR =
+      
GlobalConfiguration.loadConfiguration().getString(MetricOptions.SCOPE_DELIMITER);
 
   private final RuntimeContext runtimeContext;
   private final Map<String, Counter> flinkCounterCache;
@@ -94,7 +96,7 @@ void updateMetrics(String stepName) {
 
   private void updateCounters(Iterable<MetricResult<Long>> counters) {
     for (MetricResult<Long> metricResult : counters) {
-      String flinkMetricName = getFlinkMetricNameString(COUNTER_PREFIX, 
metricResult);
+      String flinkMetricName = getFlinkMetricNameString(metricResult);
 
       Long update = metricResult.getAttempted();
 
@@ -109,7 +111,7 @@ private void updateCounters(Iterable<MetricResult<Long>> 
counters) {
 
   private void updateDistributions(Iterable<MetricResult<DistributionResult>> 
distributions) {
     for (MetricResult<DistributionResult> metricResult : distributions) {
-      String flinkMetricName = getFlinkMetricNameString(DISTRIBUTION_PREFIX, 
metricResult);
+      String flinkMetricName = getFlinkMetricNameString(metricResult);
 
       DistributionResult update = metricResult.getAttempted();
 
@@ -129,7 +131,7 @@ private void 
updateDistributions(Iterable<MetricResult<DistributionResult>> dist
 
   private void updateGauge(Iterable<MetricResult<GaugeResult>> gauges) {
     for (MetricResult<GaugeResult> metricResult : gauges) {
-      String flinkMetricName = getFlinkMetricNameString(GAUGE_PREFIX, 
metricResult);
+      String flinkMetricName = getFlinkMetricNameString(metricResult);
 
       GaugeResult update = metricResult.getAttempted();
 
@@ -144,14 +146,13 @@ private void 
updateGauge(Iterable<MetricResult<GaugeResult>> gauges) {
     }
   }
 
-  private static String getFlinkMetricNameString(String prefix, 
MetricResult<?> metricResult) {
-    return prefix
-        + METRIC_KEY_SEPARATOR
-        + metricResult.getStep()
-        + METRIC_KEY_SEPARATOR
-        + metricResult.getName().getNamespace()
-        + METRIC_KEY_SEPARATOR
-        + metricResult.getName().getName();
+  @VisibleForTesting
+  static String getFlinkMetricNameString(MetricResult<?> metricResult) {
+    MetricName metricName = metricResult.getName();
+    // We use only the MetricName here, the step name is already contained
+    // in the operator name which is passed to Flink's MetricGroup to which
+    // the metric with the following name will be added.
+    return metricName.getNamespace() + METRIC_KEY_SEPARATOR + 
metricName.getName();
   }
 
   /** Flink {@link Gauge} for {@link DistributionResult}. */
diff --git 
a/runners/flink/src/test/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerTest.java
 
b/runners/flink/src/test/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerTest.java
new file mode 100644
index 000000000000..eda7d60959d2
--- /dev/null
+++ 
b/runners/flink/src/test/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerTest.java
@@ -0,0 +1,134 @@
+/*
+ * 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.beam.runners.flink.metrics;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Matchers.anyObject;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.core.metrics.MetricsContainerStepMap;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Distribution;
+import org.apache.beam.sdk.metrics.DistributionResult;
+import org.apache.beam.sdk.metrics.Gauge;
+import org.apache.beam.sdk.metrics.GaugeResult;
+import org.apache.beam.sdk.metrics.MetricName;
+import org.apache.beam.sdk.metrics.MetricResult;
+import org.apache.beam.sdk.metrics.MetricsContainer;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.SimpleCounter;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.mockito.MockitoAnnotations;
+
+/** Tests for {@link FlinkMetricContainer}. */
+public class FlinkMetricContainerTest {
+
+  @Mock private RuntimeContext runtimeContext;
+  @Mock private MetricGroup metricGroup;
+
+  @Before
+  public void beforeTest() {
+    MockitoAnnotations.initMocks(this);
+    when(runtimeContext.<MetricsContainerStepMap, 
MetricsContainerStepMap>getAccumulator(
+            anyString()))
+        .thenReturn(new MetricsAccumulator());
+    when(runtimeContext.getMetricGroup()).thenReturn(metricGroup);
+  }
+
+  @Test
+  public void testMetricNameGeneration() {
+    MetricResult mock = Mockito.mock(MetricResult.class);
+    when(mock.getStep()).thenReturn("step");
+    MetricName metricName = MetricName.named("namespace", "name");
+    when(mock.getName()).thenReturn(metricName);
+
+    String name = FlinkMetricContainer.getFlinkMetricNameString(mock);
+    assertThat(name, is("namespace.name"));
+  }
+
+  @Test
+  public void testCounter() {
+    SimpleCounter flinkCounter = new SimpleCounter();
+    when(metricGroup.counter("namespace.name")).thenReturn(flinkCounter);
+
+    FlinkMetricContainer container = new FlinkMetricContainer(runtimeContext);
+    MetricsContainer step = container.getMetricsContainer("step");
+    MetricName metricName = MetricName.named("namespace", "name");
+    Counter counter = step.getCounter(metricName);
+    counter.inc();
+    counter.inc();
+
+    assertThat(flinkCounter.getCount(), is(0L));
+    container.updateMetrics("step");
+    assertThat(flinkCounter.getCount(), is(2L));
+  }
+
+  @Test
+  public void testGauge() {
+    FlinkMetricContainer.FlinkGauge flinkGauge =
+        new FlinkMetricContainer.FlinkGauge(GaugeResult.empty());
+    when(metricGroup.gauge(eq("namespace.name"), 
anyObject())).thenReturn(flinkGauge);
+
+    FlinkMetricContainer container = new FlinkMetricContainer(runtimeContext);
+    MetricsContainer step = container.getMetricsContainer("step");
+    MetricName metricName = MetricName.named("namespace", "name");
+    Gauge gauge = step.getGauge(metricName);
+
+    assertThat(flinkGauge.getValue(), is(GaugeResult.empty()));
+    // first set will install the mocked gauge
+    container.updateMetrics("step");
+    gauge.set(1);
+    gauge.set(42);
+    container.updateMetrics("step");
+    assertThat(flinkGauge.getValue().getValue(), is(42L));
+  }
+
+  @Test
+  public void testDistribution() {
+    FlinkMetricContainer.FlinkDistributionGauge flinkGauge =
+        new 
FlinkMetricContainer.FlinkDistributionGauge(DistributionResult.IDENTITY_ELEMENT);
+    when(metricGroup.gauge(eq("namespace.name"), 
anyObject())).thenReturn(flinkGauge);
+
+    FlinkMetricContainer container = new FlinkMetricContainer(runtimeContext);
+    MetricsContainer step = container.getMetricsContainer("step");
+    MetricName metricName = MetricName.named("namespace", "name");
+    Distribution distribution = step.getDistribution(metricName);
+
+    assertThat(flinkGauge.getValue(), is(DistributionResult.IDENTITY_ELEMENT));
+    // first set will install the mocked distribution
+    container.updateMetrics("step");
+    distribution.update(42);
+    distribution.update(-23);
+    distribution.update(0);
+    distribution.update(1);
+    container.updateMetrics("step");
+    assertThat(flinkGauge.getValue().getMax(), is(42L));
+    assertThat(flinkGauge.getValue().getMin(), is(-23L));
+    assertThat(flinkGauge.getValue().getCount(), is(4L));
+    assertThat(flinkGauge.getValue().getSum(), is(20L));
+    assertThat(flinkGauge.getValue().getMean(), is(5.0));
+  }
+}


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 174036)
    Time Spent: 2.5h  (was: 2h 20m)

> Flink metrics are not generated in standard format
> --------------------------------------------------
>
>                 Key: BEAM-6172
>                 URL: https://issues.apache.org/jira/browse/BEAM-6172
>             Project: Beam
>          Issue Type: Improvement
>          Components: runner-flink
>    Affects Versions: 2.8.0
>            Reporter: Micah Wylde
>            Assignee: Maximilian Michels
>            Priority: Minor
>          Time Spent: 2.5h
>  Remaining Estimate: 0h
>
> The metrics that the flink runner exports do not follow the standard format 
> used by Flink, and doesn't respect Flink metric configuration options. 
> For example (with the default metrics configuration) beam produces a metric:
> {code}
> 10-100-209-71.taskmanager.0f29b420b63fea58f6f321bc0cbf45f3.BeamApp-mwylde-1203224439-a7d8fdf6.group.0.__counter__group__org-apache-beam-runners-core-ReduceFnRunner__droppedDueToClosedWindow
> {code}
> whereas a native Flink metric looks like:
> {code}
> 10-100-209-71.taskmanager.0f29b420b63fea58f6f321bc0cbf45f3.BeamApp-mwylde-1203224439-a7d8fdf6.Source-Custom-Source-7Kinesis-None-beam-env-docker-v1-0-ToKeyedWorkItem.0.numRecordsOut
> {code}
> In particular, Beam should respect the 
> [metric.scope.delimiter|https://ci.apache.org/projects/flink/flink-docs-release-1.5/ops/config.html#metrics-scope-delimiter]
>  configuration for separating components of a metric (currently it uses 
> "__"), and should not include the type of metric (counter, gauge, etc.).



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to