36
36
import org .apache .beam .runners .core .metrics .DistributionData ;
37
37
import org .apache .beam .runners .core .metrics .GaugeCell ;
38
38
import org .apache .beam .runners .core .metrics .MetricsMap ;
39
+ import org .apache .beam .runners .core .metrics .MonitoringInfoConstants ;
39
40
import org .apache .beam .runners .core .metrics .StringSetCell ;
40
41
import org .apache .beam .runners .core .metrics .StringSetData ;
41
42
import org .apache .beam .sdk .metrics .BoundedTrie ;
@@ -75,9 +76,6 @@ public class StreamingStepMetricsContainer implements MetricsContainer {
75
76
76
77
private MetricsMap <MetricName , GaugeCell > gauges = new MetricsMap <>(GaugeCell ::new );
77
78
78
- private final ConcurrentHashMap <MetricName , GaugeCell > perWorkerGauges =
79
- new ConcurrentHashMap <>();
80
-
81
79
private MetricsMap <MetricName , StringSetCell > stringSets = new MetricsMap <>(StringSetCell ::new );
82
80
83
81
private MetricsMap <MetricName , DeltaDistributionCell > distributions =
@@ -178,19 +176,6 @@ public Gauge getGauge(MetricName metricName) {
178
176
return gauges .get (metricName );
179
177
}
180
178
181
- @ Override
182
- public Gauge getPerWorkerGauge (MetricName metricName ) {
183
- if (!enablePerWorkerMetrics ) {
184
- return MetricsContainer .super .getPerWorkerGauge (metricName );
185
- }
186
- Gauge val = perWorkerGauges .get (metricName );
187
- if (val != null ) {
188
- return val ;
189
- }
190
-
191
- return perWorkerGauges .computeIfAbsent (metricName , name -> new GaugeCell (metricName ));
192
- }
193
-
194
179
@ Override
195
180
public StringSet getStringSet (MetricName metricName ) {
196
181
return stringSets .get (metricName );
@@ -256,10 +241,15 @@ private FluentIterable<CounterUpdate> gaugeUpdates() {
256
241
@ Override
257
242
public @ Nullable CounterUpdate apply (
258
243
@ Nonnull Map .Entry <MetricName , GaugeCell > entry ) {
259
- long value = entry .getValue ().getCumulative ().value ();
260
- org .joda .time .Instant timestamp = entry .getValue ().getCumulative ().timestamp ();
261
- return MetricsToCounterUpdateConverter .fromGauge (
262
- MetricKey .create (stepName , entry .getKey ()), value , timestamp );
244
+ if (!MonitoringInfoConstants .isPerWorkerMetric (entry .getKey ())) {
245
+ long value = entry .getValue ().getCumulative ().value ();
246
+ org .joda .time .Instant timestamp = entry .getValue ().getCumulative ().timestamp ();
247
+ return MetricsToCounterUpdateConverter .fromGauge (
248
+ MetricKey .create (stepName , entry .getKey ()), value , timestamp );
249
+ } else {
250
+ // add a test for this.
251
+ return null ;
252
+ }
263
253
}
264
254
})
265
255
.filter (Predicates .notNull ());
@@ -388,7 +378,8 @@ private void deleteStaleCounters(
388
378
@ VisibleForTesting
389
379
Iterable <PerStepNamespaceMetrics > extractPerWorkerMetricUpdates () {
390
380
ConcurrentHashMap <MetricName , Long > counters = new ConcurrentHashMap <MetricName , Long >();
391
- ConcurrentHashMap <MetricName , Long > gauges = new ConcurrentHashMap <MetricName , Long >();
381
+ ConcurrentHashMap <MetricName , Long > per_worker_gauges =
382
+ new ConcurrentHashMap <MetricName , Long >();
392
383
ConcurrentHashMap <MetricName , LockFreeHistogram .Snapshot > histograms =
393
384
new ConcurrentHashMap <MetricName , LockFreeHistogram .Snapshot >();
394
385
HashSet <MetricName > currentZeroValuedCounters = new HashSet <MetricName >();
@@ -402,12 +393,18 @@ Iterable<PerStepNamespaceMetrics> extractPerWorkerMetricUpdates() {
402
393
counters .put (k , val );
403
394
});
404
395
405
- perWorkerGauges .forEach (
396
+ gauges .forEach (
406
397
(k , v ) -> {
407
- Long val = v .getCumulative ().value ();
408
- gauges .put (k , val );
409
- v .reset ();
398
+ // Check if metric name has the per worker label set.
399
+ // TODO(Naireen): Populate local map with perWorkerMetrics so we don't need to check each
400
+ // time we update the metrics.
401
+ if (MonitoringInfoConstants .isPerWorkerMetric (k )) {
402
+ Long val = v .getCumulative ().value ();
403
+ per_worker_gauges .put (k , val );
404
+ v .reset ();
405
+ }
410
406
});
407
+
411
408
perWorkerHistograms .forEach (
412
409
(k , v ) -> {
413
410
v .getSnapshotAndReset ().ifPresent (snapshot -> histograms .put (k , snapshot ));
@@ -416,7 +413,7 @@ Iterable<PerStepNamespaceMetrics> extractPerWorkerMetricUpdates() {
416
413
deleteStaleCounters (currentZeroValuedCounters , Instant .now (clock ));
417
414
418
415
return MetricsToPerStepNamespaceMetricsConverter .convert (
419
- stepName , counters , gauges , histograms , parsedPerWorkerMetricsCache );
416
+ stepName , counters , per_worker_gauges , histograms , parsedPerWorkerMetricsCache );
420
417
}
421
418
422
419
/**
0 commit comments