Skip to content

Commit

Permalink
add counter stuff
Browse files Browse the repository at this point in the history
  • Loading branch information
Naireen committed Dec 10, 2024
1 parent 6edfcd4 commit be7242c
Show file tree
Hide file tree
Showing 10 changed files with 372 additions and 14 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -84,6 +84,32 @@ private static Optional<MetricValue> convertCounterToMetricValue(
.setValueInt64(value));
}

/**
* @param metricName The {@link MetricName} that represents this counter.
* @param value The counter value.
* @return If the conversion succeeds, {@code MetricValue} that represents this counter. Otherwise
* returns an empty optional
*/
private static Optional<MetricValue> convertGaugeToMetricValue(
MetricName metricName,
Long value,
Map<MetricName, LabeledMetricNameUtils.ParsedMetricName> parsedPerWorkerMetricsCache) {

if ((!metricName.getNamespace().equals(BigQuerySinkMetrics.METRICS_NAMESPACE)
&& !metricName.getNamespace().equals(KafkaSinkMetrics.METRICS_NAMESPACE))) {
return Optional.empty();
}

return getParsedMetricName(metricName, parsedPerWorkerMetricsCache)
.filter(labeledName -> !labeledName.getBaseName().isEmpty())
.map(
labeledName ->
new MetricValue()
.setMetric(labeledName.getBaseName())
.setMetricLabels(labeledName.getMetricLabels())
.setGaugeInt64(value)); // change this
}

/**
* Adds {@code outlierStats} to {@code outputHistogram} if {@code inputHistogram} has recorded
* overflow or underflow values.
Expand Down Expand Up @@ -194,6 +220,7 @@ private static Optional<MetricValue> convertHistogramToMetricValue(
public static Collection<PerStepNamespaceMetrics> convert(
String stepName,
Map<MetricName, Long> counters,
Map<MetricName, Long> gauges,
Map<MetricName, LockFreeHistogram.Snapshot> histograms,
Map<MetricName, LabeledMetricNameUtils.ParsedMetricName> parsedPerWorkerMetricsCache) {

Expand Down Expand Up @@ -243,6 +270,27 @@ public static Collection<PerStepNamespaceMetrics> convert(
stepNamespaceMetrics.getMetricValues().add(metricValue.get());
}

for (Entry<MetricName, Long> entry : gauges.entrySet()) {
MetricName metricName = entry.getKey();
Optional<MetricValue> metricValue =
convertGaugeToMetricValue(metricName, entry.getValue(), parsedPerWorkerMetricsCache);
if (!metricValue.isPresent()) {
continue;
}

PerStepNamespaceMetrics stepNamespaceMetrics =
metricsByNamespace.get(metricName.getNamespace());
if (stepNamespaceMetrics == null) {
stepNamespaceMetrics =
new PerStepNamespaceMetrics()
.setMetricValues(new ArrayList<>())
.setOriginalStep(stepName)
.setMetricsNamespace(metricName.getNamespace());
metricsByNamespace.put(metricName.getNamespace(), stepNamespaceMetrics);
}

stepNamespaceMetrics.getMetricValues().add(metricValue.get());
}
return metricsByNamespace.values();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -665,6 +665,7 @@ public static void main(String[] args) throws Exception {
}

if (DataflowRunner.hasExperiment(options, "enable_kafka_metrics")) {
LOG.info("xxx enable experiement");
KafkaSinkMetrics.setSupportKafkaMetrics(true);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,8 @@
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Predicates;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.FluentIterable;
import org.checkerframework.checker.nullness.qual.Nullable;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* For Dataflow Streaming, we want to efficiently support many threads report metric updates, and a
* single total delta being reported periodically as physical counters.
Expand All @@ -58,6 +59,9 @@
"nullness" // TODO(https://github.com/apache/beam/issues/20497)
})
public class StreamingStepMetricsContainer implements MetricsContainer {

private static final Logger LOG = LoggerFactory.getLogger(StreamingStepMetricsContainer.class);

private final String stepName;

private static boolean enablePerWorkerMetrics = false;
Expand All @@ -69,6 +73,9 @@ public class StreamingStepMetricsContainer implements MetricsContainer {

private MetricsMap<MetricName, GaugeCell> gauges = new MetricsMap<>(GaugeCell::new);

// how to handle concurrency
private final ConcurrentHashMap<MetricName, GaugeCell> perWorkerGauges = new ConcurrentHashMap<>();

private MetricsMap<MetricName, StringSetCell> stringSet = new MetricsMap<>(StringSetCell::new);

private MetricsMap<MetricName, DeltaDistributionCell> distributions =
Expand All @@ -82,7 +89,7 @@ public class StreamingStepMetricsContainer implements MetricsContainer {
private final ConcurrentHashMap<MetricName, LabeledMetricNameUtils.ParsedMetricName>
parsedPerWorkerMetricsCache;

// PerWorkerCounters that have been longer than this value will be removed from the underlying
// PerWorkerCounters and PerWorkerGauges that have been longer than this value will be removed from the underlying
// metrics map.
private final Duration maximumPerWorkerCounterStaleness = Duration.ofMinutes(5);

Expand Down Expand Up @@ -163,6 +170,21 @@ public Gauge getGauge(MetricName metricName) {
return gauges.get(metricName);
}

@Override
public Gauge getPerWorkerGauge(MetricName metricName) {
if (!enablePerWorkerMetrics) {
return MetricsContainer.super.getPerWorkerGauge(metricName);
}

Gauge val = perWorkerGauges.get(metricName);
if (val != null) {
return val;
}

return perWorkerGauges.computeIfAbsent(
metricName, name -> new GaugeCell(metricName));
}

@Override
public StringSet getStringSet(MetricName metricName) {
return stringSet.get(metricName);
Expand Down Expand Up @@ -330,10 +352,11 @@ private void deleteStaleCounters(
@VisibleForTesting
Iterable<PerStepNamespaceMetrics> extractPerWorkerMetricUpdates() {
ConcurrentHashMap<MetricName, Long> counters = new ConcurrentHashMap<MetricName, Long>();
ConcurrentHashMap<MetricName, Long> gauges = new ConcurrentHashMap<MetricName, Long>();
ConcurrentHashMap<MetricName, LockFreeHistogram.Snapshot> histograms =
new ConcurrentHashMap<MetricName, LockFreeHistogram.Snapshot>();
HashSet<MetricName> currentZeroValuedCounters = new HashSet<MetricName>();

LOG.info("xxx per worker extractPerWorkerMetricUpdates");
// Extract metrics updates.
perWorkerCounters.forEach(
(k, v) -> {
Expand All @@ -344,6 +367,12 @@ Iterable<PerStepNamespaceMetrics> extractPerWorkerMetricUpdates() {
}
counters.put(k, val);
});
perWorkerGauges.forEach(
(k, v) -> {
Long val = v.getCumulative().value();
LOG.info("xxx per worker gauges k {} val {}", k.getName(), val);
gauges.put(k, val); // no special handing for zero, since that value is important
});
perWorkerHistograms.forEach(
(k, v) -> {
v.getSnapshotAndReset().ifPresent(snapshot -> histograms.put(k, snapshot));
Expand All @@ -352,7 +381,7 @@ Iterable<PerStepNamespaceMetrics> extractPerWorkerMetricUpdates() {
deleteStaleCounters(currentZeroValuedCounters, Instant.now(clock));

return MetricsToPerStepNamespaceMetricsConverter.convert(
stepName, counters, histograms, parsedPerWorkerMetricsCache);
stepName, counters, gauges, histograms, parsedPerWorkerMetricsCache);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,7 @@ public void testConvert_successfulyConvertCounters() {

Collection<PerStepNamespaceMetrics> conversionResult =
MetricsToPerStepNamespaceMetricsConverter.convert(
step, counters, emptyHistograms, parsedMetricNames);
step, counters, counters, emptyHistograms, parsedMetricNames);

MetricValue expectedVal1 =
new MetricValue().setMetric("metric1").setValueInt64(5L).setMetricLabels(new HashMap<>());
Expand Down Expand Up @@ -144,7 +144,7 @@ public void testConvert_skipInvalidMetricNames() {

Collection<PerStepNamespaceMetrics> conversionResult =
MetricsToPerStepNamespaceMetricsConverter.convert(
"testStep", counters, histograms, parsedMetricNames);
"testStep", counters, counters, histograms, parsedMetricNames);
assertThat(conversionResult.size(), equalTo(0));
assertThat(parsedMetricNames.size(), equalTo(0));
}
Expand Down Expand Up @@ -181,7 +181,7 @@ public void testConvert_successfulConvertHistograms() {
Map<MetricName, Long> emptyCounters = new HashMap<>();
Collection<PerStepNamespaceMetrics> conversionResult =
MetricsToPerStepNamespaceMetricsConverter.convert(
step, emptyCounters, histograms, parsedMetricNames);
step, emptyCounters,emptyCounters, histograms, parsedMetricNames);

// Expected value 1
List<Long> bucketCounts1 = ImmutableList.of(0L, 1L, 1L, 1L);
Expand Down Expand Up @@ -271,7 +271,7 @@ public void testConvert_skipUnknownHistogramBucketType() {

Collection<PerStepNamespaceMetrics> conversionResult =
MetricsToPerStepNamespaceMetricsConverter.convert(
step, emptyCounters, histograms, parsedMetricNames);
step, emptyCounters, emptyCounters, histograms, parsedMetricNames);
assertThat(conversionResult.size(), equalTo(0));
assertThat(parsedMetricNames.size(), equalTo(0));
}
Expand All @@ -293,7 +293,7 @@ public void testConvert_convertCountersAndHistograms() {

Collection<PerStepNamespaceMetrics> conversionResult =
MetricsToPerStepNamespaceMetricsConverter.convert(
step, counters, histograms, parsedMetricNames);
step, counters, counters, histograms, parsedMetricNames);

// Expected counter MetricValue
Map<String, String> counterLabelMap = new HashMap<>();
Expand Down Expand Up @@ -345,4 +345,53 @@ public void testConvert_convertCountersAndHistograms() {
parsedMetricNames,
IsMapContaining.hasEntry(histogramMetricName, parsedHistogramMetricName));
}

// @Test
// public void testConvert_successfulyConvertGauges() {
// String step = "testStepName";
// Map<MetricName, LockFreeHistogram.Snapshot> emptyHistograms = new HashMap<>();
// Map<MetricName, Long> counters = new HashMap<MetricName, Long>();
// Map<MetricName, Long> gauge = new HashMap<MetricName, Long>();

// Map<MetricName, LabeledMetricNameUtils.ParsedMetricName> parsedMetricNames = new HashMap<>();

// MetricName bigQueryMetric1 = MetricName.named("KafkaSink", "metric1");
// MetricName bigQueryMetric2 =
// MetricName.named("KafkaSink", "metric2*label1:val1;label2:val2;");
// MetricName bigQueryMetric3 = MetricName.named("BigQuerySink", "zeroValue"); // ?

// counters.put(bigQueryMetric1, 5L);
// counters.put(bigQueryMetric2, 10L);
// counters.put(bigQueryMetric3, 0L);

// Collection<PerStepNamespaceMetrics> conversionResult =
// MetricsToPerStepNamespaceMetricsConverter.convert(
// step, counters, emptyHistograms, parsedMetricNames);

// MetricValue expectedVal1 =
// new MetricValue().setMetric("metric1").setGaugeValue(5L).setMetricLabels(new HashMap<>());
// Map<String, String> val2LabelMap = new HashMap<>();
// val2LabelMap.put("label1", "val1");
// val2LabelMap.put("label2", "val2");
// MetricValue expectedVal2 =
// new MetricValue().setMetric("metric2").setValueInt64(10L).setMetricLabels(val2LabelMap);

// assertThat(conversionResult.size(), equalTo(1));
// PerStepNamespaceMetrics perStepNamespaceMetrics = conversionResult.iterator().next();

// assertThat(perStepNamespaceMetrics.getOriginalStep(), equalTo(step));
// assertThat(perStepNamespaceMetrics.getMetricsNamespace(), equalTo("BigQuerySink"));
// assertThat(perStepNamespaceMetrics.getMetricValues().size(), equalTo(2));
// assertThat(
// perStepNamespaceMetrics.getMetricValues(), containsInAnyOrder(expectedVal1, expectedVal2));

// LabeledMetricNameUtils.ParsedMetricName parsedBigQueryMetric1 =
// LabeledMetricNameUtils.parseMetricName(bigQueryMetric1.getName()).get();
// LabeledMetricNameUtils.ParsedMetricName parsedBigQueryMetric2 =
// LabeledMetricNameUtils.parseMetricName(bigQueryMetric2.getName()).get();

// assertThat(parsedMetricNames.size(), equalTo(2));
// assertThat(parsedMetricNames, IsMapContaining.hasEntry(bigQueryMetric1, parsedBigQueryMetric1));
// assertThat(parsedMetricNames, IsMapContaining.hasEntry(bigQueryMetric2, parsedBigQueryMetric2));
// }
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,84 @@
/*
* 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.sdk.metrics;

import java.io.Serializable;
import org.apache.beam.sdk.annotations.Internal;

/** Implementation of {@link Gauge} that delegates to the instance for the current context. */
@Internal
public class DelegatingGauge implements Metric, Gauge, Serializable {
private final MetricName name;
private final boolean processWideContainer;
private final boolean perWorkerGauge;

/**
* Create a {@code DelegatingGauge} with {@code perWorkerGauge} and {@code processWideContainer}
* set to false.
*
* @param name Metric name for this metric.
*/
public DelegatingGauge(MetricName name) {
this(name, false, false);
}

/**
* Create a {@code DelegatingGauge} with {@code perWorkerGauge} set to false.
*
* @param name Metric name for this metric.
* @param processWideContainer Whether this Gauge is stored in the ProcessWide container or the
* current thread's container.
*/
public DelegatingGauge(MetricName name, boolean processWideContainer) {
this(name, processWideContainer, false);
}

/**
* @param name Metric name for this metric.
* @param processWideContainer Whether this gauge is stored in the ProcessWide container or the
* current thread's container.
* @param perWorkerGauge Whether this gauge refers to a perWorker metric or not.
*/
public DelegatingGauge(MetricName name, boolean processWideContainer, boolean perWorkerGauge) {
this.name = name;
this.processWideContainer = processWideContainer;
this.perWorkerGauge = perWorkerGauge;
}

/** Set the gauge. */
@Override
public void set(long n) {
MetricsContainer container =
this.processWideContainer
? MetricsEnvironment.getProcessWideContainer()
: MetricsEnvironment.getCurrentContainer();
if (container == null) {
return;
}
if (perWorkerGauge) {
container.getPerWorkerGauge(name).set(n);
} else {
container.getGauge(name).set(n);
}
}

@Override
public MetricName getName() {
return name;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,13 @@ default Histogram getPerWorkerHistogram(
return NoOpHistogram.getInstance();
}

/**
* Return the {@link Gauge} that should be used for implementing the given per-worker {@code
* metricName} in this container.
*/
default Gauge getPerWorkerGauge(MetricName metricName) {
return NoOpGauge.getInstance();
}
/** Return the cumulative values for any metrics in this container as MonitoringInfos. */
default Iterable<MetricsApi.MonitoringInfo> getMonitoringInfos() {
throw new RuntimeException("getMonitoringInfos is not implemented on this MetricsContainer.");
Expand Down
Loading

0 comments on commit be7242c

Please sign in to comment.