-
Notifications
You must be signed in to change notification settings - Fork 4.4k
Kafka add counters v1 uw2 #33503
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Kafka add counters v1 uw2 #33503
Changes from all commits
4c39d3a
6c42cf4
301aa6a
8620b36
1c582f5
8e7785b
8e89e97
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -36,6 +36,7 @@ | |
import org.apache.beam.runners.core.metrics.DistributionData; | ||
import org.apache.beam.runners.core.metrics.GaugeCell; | ||
import org.apache.beam.runners.core.metrics.MetricsMap; | ||
import org.apache.beam.runners.core.metrics.MonitoringInfoConstants; | ||
import org.apache.beam.runners.core.metrics.StringSetCell; | ||
import org.apache.beam.runners.core.metrics.StringSetData; | ||
import org.apache.beam.sdk.metrics.BoundedTrie; | ||
|
@@ -75,9 +76,6 @@ public class StreamingStepMetricsContainer implements MetricsContainer { | |
|
||
private MetricsMap<MetricName, GaugeCell> gauges = new MetricsMap<>(GaugeCell::new); | ||
|
||
private final ConcurrentHashMap<MetricName, GaugeCell> perWorkerGauges = | ||
new ConcurrentHashMap<>(); | ||
|
||
private MetricsMap<MetricName, StringSetCell> stringSets = new MetricsMap<>(StringSetCell::new); | ||
|
||
private MetricsMap<MetricName, DeltaDistributionCell> distributions = | ||
|
@@ -178,19 +176,6 @@ 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 stringSets.get(metricName); | ||
|
@@ -256,10 +241,15 @@ private FluentIterable<CounterUpdate> gaugeUpdates() { | |
@Override | ||
public @Nullable CounterUpdate apply( | ||
@Nonnull Map.Entry<MetricName, GaugeCell> entry) { | ||
long value = entry.getValue().getCumulative().value(); | ||
org.joda.time.Instant timestamp = entry.getValue().getCumulative().timestamp(); | ||
return MetricsToCounterUpdateConverter.fromGauge( | ||
MetricKey.create(stepName, entry.getKey()), value, timestamp); | ||
if (!MonitoringInfoConstants.isPerWorkerMetric(entry.getKey())) { | ||
long value = entry.getValue().getCumulative().value(); | ||
org.joda.time.Instant timestamp = entry.getValue().getCumulative().timestamp(); | ||
return MetricsToCounterUpdateConverter.fromGauge( | ||
MetricKey.create(stepName, entry.getKey()), value, timestamp); | ||
} else { | ||
// add a test for this. | ||
return null; | ||
} | ||
} | ||
}) | ||
.filter(Predicates.notNull()); | ||
|
@@ -388,7 +378,8 @@ 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, Long> per_worker_gauges = | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: I don't think this (or counters) needs to be Concurrent. How about using ImmutableMapBuilder and then build when passing to convert? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I agree, that should be done for all of them, will do that in the followup clean up CL. |
||
new ConcurrentHashMap<MetricName, Long>(); | ||
ConcurrentHashMap<MetricName, LockFreeHistogram.Snapshot> histograms = | ||
new ConcurrentHashMap<MetricName, LockFreeHistogram.Snapshot>(); | ||
HashSet<MetricName> currentZeroValuedCounters = new HashSet<MetricName>(); | ||
|
@@ -402,12 +393,18 @@ Iterable<PerStepNamespaceMetrics> extractPerWorkerMetricUpdates() { | |
counters.put(k, val); | ||
}); | ||
|
||
perWorkerGauges.forEach( | ||
gauges.forEach( | ||
(k, v) -> { | ||
Long val = v.getCumulative().value(); | ||
gauges.put(k, val); | ||
v.reset(); | ||
// Check if metric name has the per worker label set. | ||
// TODO(Naireen): Populate local map with perWorkerMetrics so we don't need to check each | ||
// time we update the metrics. | ||
if (MonitoringInfoConstants.isPerWorkerMetric(k)) { | ||
Long val = v.getCumulative().value(); | ||
per_worker_gauges.put(k, val); | ||
v.reset(); | ||
} | ||
}); | ||
|
||
perWorkerHistograms.forEach( | ||
(k, v) -> { | ||
v.getSnapshotAndReset().ifPresent(snapshot -> histograms.put(k, snapshot)); | ||
|
@@ -416,7 +413,7 @@ Iterable<PerStepNamespaceMetrics> extractPerWorkerMetricUpdates() { | |
deleteStaleCounters(currentZeroValuedCounters, Instant.now(clock)); | ||
|
||
return MetricsToPerStepNamespaceMetricsConverter.convert( | ||
stepName, counters, gauges, histograms, parsedPerWorkerMetricsCache); | ||
stepName, counters, per_worker_gauges, histograms, parsedPerWorkerMetricsCache); | ||
} | ||
|
||
/** | ||
|
Uh oh!
There was an error while loading. Please reload this page.