Skip to content
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

Introduce a BoundedTrie metric which is used to efficiently store and… #33385

Merged
merged 22 commits into from
Feb 1, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
22 commits
Select commit Hold shift + click to select a range
0ba6c8a
Introduce a BoundedTrie metric which is used to efficiently store and…
rohitsinha54 Dec 16, 2024
b476466
Address comments, perform deep copies and support synchronization for…
rohitsinha54 Dec 19, 2024
7849f5f
Add more unit tests for trienode and fix merge on empty node
rohitsinha54 Dec 20, 2024
aaf3bd8
Add more unit test for BoundedTrieData which are not already covered …
rohitsinha54 Dec 20, 2024
2b41b17
Add unit tests for Cell
rohitsinha54 Dec 20, 2024
631a34b
Plumb BoundedTrie to MetricsContainerImpl
rohitsinha54 Dec 21, 2024
ba85636
Plumb BoundedTrie to StreamingStepMetricsContainer
rohitsinha54 Dec 22, 2024
a60330f
Plumb BoundedTrie to StreamingStepMetricsContainer
rohitsinha54 Dec 22, 2024
5e2ec9e
Plumb BoundedTrie to MetricQueryResults
rohitsinha54 Dec 23, 2024
b7b74e6
Plumb BoundedTrie to MetricsContainerStepMap and make combine return …
rohitsinha54 Dec 23, 2024
6cef4c5
Add test in MetricsContainerStepMapTest
rohitsinha54 Dec 23, 2024
6cd8fdc
Plumb BoundedTrie to DirectMetrics
rohitsinha54 Dec 23, 2024
a7e237e
Add BoundedTrieResult class
rohitsinha54 Dec 24, 2024
4825bf2
Plumb BoundedTrieResult and BoundedTrie in DataflowMetrics
rohitsinha54 Dec 25, 2024
5908879
Plumb BoundedTrie JetMetric and PortableMetric
rohitsinha54 Dec 25, 2024
78508bf
Add test for BoundedTrieResult and some more plumbing
rohitsinha54 Dec 25, 2024
c959101
Fixes for github checks
rohitsinha54 Dec 25, 2024
3761e59
Multi-threaded test in MetricsContainerImplTest and some clean up
rohitsinha54 Dec 27, 2024
35ea66b
Address comments part 2
rohitsinha54 Dec 31, 2024
b9d3986
Drop support DataflowMetrics and MetricsToCounterUpdateConverter unti…
rohitsinha54 Dec 31, 2024
31b41b0
rebase fixes
rohitsinha54 Jan 30, 2025
f68fe75
Suppress IS2_INCONSISTENT_SYNC warning
rohitsinha54 Feb 1, 2025
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,108 @@
/*
* 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.runners.core.metrics;

import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
import java.util.Arrays;
import java.util.Objects;
import org.apache.beam.sdk.metrics.BoundedTrie;
import org.apache.beam.sdk.metrics.MetricName;
import org.checkerframework.checker.nullness.qual.Nullable;

/**
* Tracks the current value for a {@link BoundedTrie} metric.
*
* <p>This class generally shouldn't be used directly. The only exception is within a runner where a
* counter is being reported for a specific step (rather than the counter in the current context).
* In that case retrieving the underlying cell and reporting directly to it avoids a step of
* indirection.
*/
@SuppressFBWarnings(
value = "IS2_INCONSISTENT_SYNC",
justification = "Some access on purpose are left unsynchronized")
public class BoundedTrieCell implements BoundedTrie, MetricCell<BoundedTrieData> {

private final DirtyState dirty = new DirtyState();
private BoundedTrieData value;
private final MetricName name;

public BoundedTrieCell(MetricName name) {
this.name = name;
this.value = new BoundedTrieData();
}

public synchronized void update(BoundedTrieData other) {
// although BoundedTrieData is thread-safe the cell is made thread safe too because combine
// returns a reference to the combined BoundedTrieData and want the reference update here to
// be thread safe too.
this.value = this.value.combine(other);
dirty.afterModification();
}

@Override
public synchronized void reset() {
value.clear();
dirty.reset();
}

@Override
public DirtyState getDirty() {
return dirty;
}

/**
* @return Returns a deep copy of the {@link BoundedTrieData} contained in this {@link
* BoundedTrieCell}.
*/
@Override
public synchronized BoundedTrieData getCumulative() {
return value.getCumulative();
}

@Override
public MetricName getName() {
return name;
}

@Override
public boolean equals(@Nullable Object object) {
if (object instanceof BoundedTrieCell) {
BoundedTrieCell boundedTrieCell = (BoundedTrieCell) object;
return Objects.equals(dirty, boundedTrieCell.dirty)
&& Objects.equals(value, boundedTrieCell.value)
&& Objects.equals(name, boundedTrieCell.name);
}
return false;
}

@Override
public int hashCode() {
return Objects.hash(dirty, value, name);
}

@Override
public synchronized void add(Iterable<String> values) {
this.value.add(values);
dirty.afterModification();
}

@Override
public synchronized void add(String... values) {
add(Arrays.asList(values));
}
}
Loading
Loading