|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | +package org.apache.beam.runners.samza.runtime; |
| 19 | + |
| 20 | +import static org.apache.beam.runners.core.metrics.MonitoringInfoConstants.TypeUrns.DISTRIBUTION_INT64_TYPE; |
| 21 | +import static org.apache.beam.runners.core.metrics.MonitoringInfoConstants.TypeUrns.LATEST_INT64_TYPE; |
| 22 | +import static org.apache.beam.runners.core.metrics.MonitoringInfoConstants.TypeUrns.SUM_INT64_TYPE; |
| 23 | +import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.decodeInt64Counter; |
| 24 | +import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.decodeInt64Distribution; |
| 25 | +import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.decodeInt64Gauge; |
| 26 | + |
| 27 | +import java.util.Map; |
| 28 | +import org.apache.beam.model.fnexecution.v1.BeamFnApi; |
| 29 | +import org.apache.beam.model.pipeline.v1.MetricsApi; |
| 30 | +import org.apache.beam.runners.core.metrics.DistributionData; |
| 31 | +import org.apache.beam.runners.core.metrics.MonitoringInfoConstants; |
| 32 | +import org.apache.beam.runners.fnexecution.control.BundleProgressHandler; |
| 33 | +import org.apache.beam.runners.samza.metrics.SamzaMetricsContainer; |
| 34 | +import org.apache.beam.sdk.metrics.Counter; |
| 35 | +import org.apache.beam.sdk.metrics.Distribution; |
| 36 | +import org.apache.beam.sdk.metrics.Gauge; |
| 37 | +import org.apache.beam.sdk.metrics.MetricName; |
| 38 | +import org.apache.beam.sdk.metrics.MetricsContainer; |
| 39 | +import org.slf4j.Logger; |
| 40 | +import org.slf4j.LoggerFactory; |
| 41 | + |
| 42 | +/** |
| 43 | + * {@inheritDoc} Parses metrics information contained in the bundle progress messages. Passed the |
| 44 | + * updated metrics to the provided SamzaMetricsContainer. |
| 45 | + */ |
| 46 | +class SamzaMetricsBundleProgressHandler implements BundleProgressHandler { |
| 47 | + |
| 48 | + private static final Logger LOG = |
| 49 | + LoggerFactory.getLogger(SamzaMetricsBundleProgressHandler.class); |
| 50 | + private final String stepName; |
| 51 | + |
| 52 | + private final SamzaMetricsContainer samzaMetricsContainer; |
| 53 | + private final Map<String, String> transformIdToUniqueName; |
| 54 | + |
| 55 | + /** |
| 56 | + * Constructor of a SamzaMetricsBundleProgressHandler. |
| 57 | + * |
| 58 | + * <p>The full metric names in classic mode is {transformUniqueName}:{className}:{metricName}. We |
| 59 | + * attempt to follow the same format in portable mode, but the monitoringInfos returned by the |
| 60 | + * worker only contains the transformId. The current solution is to provide a mapping from |
| 61 | + * transformId back to uniqueName. A future improvement would be making the monitoring infos |
| 62 | + * contain the uniqueName. |
| 63 | + * |
| 64 | + * @param stepName Default stepName provided by the runner. |
| 65 | + * @param samzaMetricsContainer The destination for publishing the metrics. |
| 66 | + * @param transformIdToUniqueName A mapping from transformId to uniqueName for pTransforms. |
| 67 | + */ |
| 68 | + public SamzaMetricsBundleProgressHandler( |
| 69 | + String stepName, |
| 70 | + SamzaMetricsContainer samzaMetricsContainer, |
| 71 | + Map<String, String> transformIdToUniqueName) { |
| 72 | + this.stepName = stepName; |
| 73 | + this.samzaMetricsContainer = samzaMetricsContainer; |
| 74 | + this.transformIdToUniqueName = transformIdToUniqueName; |
| 75 | + } |
| 76 | + |
| 77 | + @Override |
| 78 | + /** |
| 79 | + * {@inheritDoc} Handles a progress report from the bundle while it is executing. We choose to |
| 80 | + * ignore the progress report. The metrics do not have to be updated on every progress report, so |
| 81 | + * we save computation resources by ignoring it. |
| 82 | + */ |
| 83 | + public void onProgress(BeamFnApi.ProcessBundleProgressResponse progress) {} |
| 84 | + |
| 85 | + @Override |
| 86 | + /** |
| 87 | + * {@inheritDoc} Handles the bundle's completion report. Parses the monitoringInfos in the |
| 88 | + * response, then updates the MetricsRegistry. |
| 89 | + */ |
| 90 | + public void onCompleted(BeamFnApi.ProcessBundleResponse response) { |
| 91 | + response.getMonitoringInfosList().stream() |
| 92 | + .filter(monitoringInfo -> !monitoringInfo.getPayload().isEmpty()) |
| 93 | + .forEach(this::parseAndUpdateMetric); |
| 94 | + } |
| 95 | + |
| 96 | + /** |
| 97 | + * Parses the metric contained in monitoringInfo, then publishes the metric to the |
| 98 | + * metricContainer. |
| 99 | + * |
| 100 | + * <p>We attempt to construct a classic mode metricName |
| 101 | + * ({transformUniqueName}:{className}:{metricName}). All the info should be in the labels, but we |
| 102 | + * have fallbacks in case the labels don't exist. |
| 103 | + * |
| 104 | + * <p>Priorities for the transformUniqueName 1. Obtained transformUniqueName using the |
| 105 | + * transformIdToUniqueName 2. The transformId provided by the monitoringInfo 3. The stepName |
| 106 | + * provided by the runner, which maybe a result of fusing. |
| 107 | + * |
| 108 | + * <p>Priorities for the className 1. The namespace label 2. The monitoringInfo urn. Copying the |
| 109 | + * implementation in MonitoringInfoMetricName. |
| 110 | + * |
| 111 | + * <p>Priorities for the metricName 1. The name label 2. The monitoringInfo urn. Copying the |
| 112 | + * implementation in MonitoringInfoMetricName. |
| 113 | + * |
| 114 | + * @see |
| 115 | + * org.apache.beam.runners.core.metrics.MonitoringInfoMetricName#of(MetricsApi.MonitoringInfo) |
| 116 | + */ |
| 117 | + private void parseAndUpdateMetric(MetricsApi.MonitoringInfo monitoringInfo) { |
| 118 | + String pTransformId = |
| 119 | + monitoringInfo.getLabelsOrDefault(MonitoringInfoConstants.Labels.PTRANSFORM, stepName); |
| 120 | + String transformUniqueName = transformIdToUniqueName.getOrDefault(pTransformId, pTransformId); |
| 121 | + String className = |
| 122 | + monitoringInfo.getLabelsOrDefault( |
| 123 | + MonitoringInfoConstants.Labels.NAMESPACE, monitoringInfo.getUrn()); |
| 124 | + String userMetricName = |
| 125 | + monitoringInfo.getLabelsOrDefault( |
| 126 | + MonitoringInfoConstants.Labels.NAME, monitoringInfo.getLabelsMap().toString()); |
| 127 | + |
| 128 | + MetricsContainer metricsContainer = samzaMetricsContainer.getContainer(transformUniqueName); |
| 129 | + MetricName metricName = MetricName.named(className, userMetricName); |
| 130 | + |
| 131 | + switch (monitoringInfo.getType()) { |
| 132 | + case SUM_INT64_TYPE: |
| 133 | + Counter counter = metricsContainer.getCounter(metricName); |
| 134 | + counter.inc(decodeInt64Counter(monitoringInfo.getPayload())); |
| 135 | + break; |
| 136 | + |
| 137 | + case DISTRIBUTION_INT64_TYPE: |
| 138 | + Distribution distribution = metricsContainer.getDistribution(metricName); |
| 139 | + DistributionData data = decodeInt64Distribution(monitoringInfo.getPayload()); |
| 140 | + distribution.update(data.sum(), data.count(), data.min(), data.max()); |
| 141 | + break; |
| 142 | + |
| 143 | + case LATEST_INT64_TYPE: |
| 144 | + Gauge gauge = metricsContainer.getGauge(metricName); |
| 145 | + // Gauge doesn't expose update as public. This will reset the timestamp. |
| 146 | + |
| 147 | + gauge.set(decodeInt64Gauge(monitoringInfo.getPayload()).value()); |
| 148 | + break; |
| 149 | + |
| 150 | + default: |
| 151 | + LOG.warn("Unsupported metric type {}", monitoringInfo.getType()); |
| 152 | + } |
| 153 | + } |
| 154 | +} |
0 commit comments