Skip to content

Commit 38ae8e9

Browse files
committed
Introduce a BoundedTrie metric which is used to efficiently store and aggregate a collection of string sequences (FQNs) with a limited size.
1 parent d9092a5 commit 38ae8e9

File tree

6 files changed

+1260
-0
lines changed

6 files changed

+1260
-0
lines changed
Lines changed: 110 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,110 @@
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.core.metrics;
19+
20+
import java.util.Arrays;
21+
import java.util.Objects;
22+
import java.util.concurrent.atomic.AtomicReference;
23+
import org.apache.beam.sdk.metrics.BoundedTrie;
24+
import org.apache.beam.sdk.metrics.MetricName;
25+
import org.apache.beam.sdk.metrics.MetricsContainer;
26+
import org.checkerframework.checker.nullness.qual.Nullable;
27+
28+
/**
29+
* Tracks the current value for a {@link BoundedTrie} metric.
30+
*
31+
* <p>This class generally shouldn't be used directly. The only exception is within a runner where a
32+
* counter is being reported for a specific step (rather than the counter in the current context).
33+
* In that case retrieving the underlying cell and reporting directly to it avoids a step of
34+
* indirection.
35+
*/
36+
public class BoundedTrieCell implements BoundedTrie, MetricCell<BoundedTrieData> {
37+
38+
private final DirtyState dirty = new DirtyState();
39+
private final AtomicReference<BoundedTrieData> setValue =
40+
new AtomicReference<>(BoundedTrieData.empty());
41+
private final MetricName name;
42+
43+
/**
44+
* Generally, runners should construct instances using the methods in {@link
45+
* MetricsContainerImpl}, unless they need to define their own version of {@link
46+
* MetricsContainer}. These constructors are *only* public so runners can instantiate.
47+
*/
48+
public BoundedTrieCell(MetricName name) {
49+
this.name = name;
50+
}
51+
52+
@Override
53+
public void reset() {
54+
setValue.set(BoundedTrieData.empty());
55+
dirty.reset();
56+
}
57+
58+
void update(BoundedTrieData data) {
59+
BoundedTrieData original;
60+
do {
61+
original = setValue.get();
62+
} while (!setValue.compareAndSet(original, original.combine(data)));
63+
dirty.afterModification();
64+
}
65+
66+
@Override
67+
public DirtyState getDirty() {
68+
return dirty;
69+
}
70+
71+
@Override
72+
public BoundedTrieData getCumulative() {
73+
return setValue.get();
74+
}
75+
76+
@Override
77+
public MetricName getName() {
78+
return name;
79+
}
80+
81+
@Override
82+
public boolean equals(@Nullable Object object) {
83+
if (object instanceof BoundedTrieCell) {
84+
BoundedTrieCell boundedTrieCell = (BoundedTrieCell) object;
85+
return Objects.equals(dirty, boundedTrieCell.dirty)
86+
&& Objects.equals(setValue.get(), boundedTrieCell.setValue.get())
87+
&& Objects.equals(name, boundedTrieCell.name);
88+
}
89+
return false;
90+
}
91+
92+
@Override
93+
public int hashCode() {
94+
return Objects.hash(dirty, setValue.get(), name);
95+
}
96+
97+
@Override
98+
public void add(Iterable<String> values) {
99+
BoundedTrieData original;
100+
do {
101+
original = setValue.get();
102+
} while (!setValue.compareAndSet(original, original.add(values)));
103+
dirty.afterModification();
104+
}
105+
106+
@Override
107+
public void add(String... values) {
108+
add(Arrays.asList(values));
109+
}
110+
}

0 commit comments

Comments
 (0)