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

Open
wants to merge 20 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 5 commits
Commits
Show all changes
20 commits
Select commit Hold shift + click to select a range
38ae8e9
Introduce a BoundedTrie metric which is used to efficiently store and…
rohitsinha54 Dec 16, 2024
35e0a8c
Address comments, perform deep copies and support synchronization for…
rohitsinha54 Dec 19, 2024
d76412a
Add more unit tests for trienode and fix merge on empty node
rohitsinha54 Dec 20, 2024
10f68d0
Add more unit test for BoundedTrieData which are not already covered …
rohitsinha54 Dec 20, 2024
ba96765
Add unit tests for Cell
rohitsinha54 Dec 20, 2024
bfe3de0
Plumb BoundedTrie to MetricsContainerImpl
rohitsinha54 Dec 21, 2024
4b0b070
Plumb BoundedTrie to StreamingStepMetricsContainer
rohitsinha54 Dec 22, 2024
3c5d613
Plumb BoundedTrie to StreamingStepMetricsContainer
rohitsinha54 Dec 22, 2024
724d310
Plumb BoundedTrie to MetricQueryResults
rohitsinha54 Dec 23, 2024
c0bb3f1
Plumb BoundedTrie to MetricsContainerStepMap and make combine return …
rohitsinha54 Dec 23, 2024
f9faf23
Add test in MetricsContainerStepMapTest
rohitsinha54 Dec 23, 2024
0c8aa32
Plumb BoundedTrie to DirectMetrics
rohitsinha54 Dec 23, 2024
aaaff29
Add BoundedTrieResult class
rohitsinha54 Dec 24, 2024
386c362
Plumb BoundedTrieResult and BoundedTrie in DataflowMetrics
rohitsinha54 Dec 25, 2024
7a2509c
Plumb BoundedTrie JetMetric and PortableMetric
rohitsinha54 Dec 25, 2024
83b3d0d
Add test for BoundedTrieResult and some more plumbing
rohitsinha54 Dec 25, 2024
269c5d5
Fixes for github checks
rohitsinha54 Dec 25, 2024
9298dfa
Multi-threaded test in MetricsContainerImplTest and some clean up
rohitsinha54 Dec 27, 2024
eea0acf
Address comments part 2
rohitsinha54 Dec 31, 2024
a2cb708
Drop support DataflowMetrics and MetricsToCounterUpdateConverter unti…
rohitsinha54 Dec 31, 2024
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,98 @@
/*
* 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 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.
*/
// TODO: Write multi-threaded test in MetricContainerImp for this Cell class too.
public class BoundedTrieCell implements BoundedTrie, MetricCell<BoundedTrieData> {

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

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

public void update(BoundedTrieData other) {
this.value.combine(other);
dirty.afterModification();
}

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

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

@Override
public 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 void add(Iterable<String> values) {
this.value.add(values);
dirty.afterModification();
}

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