Skip to content

Commit

Permalink
Address comments part 2
Browse files Browse the repository at this point in the history
  • Loading branch information
rohitsinha54 committed Dec 30, 2024
1 parent 034c4e6 commit db4e944
Show file tree
Hide file tree
Showing 2 changed files with 34 additions and 23 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
*/
package org.apache.beam.runners.core.metrics;

import com.google.common.annotations.VisibleForTesting;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collections;
Expand Down Expand Up @@ -44,8 +45,8 @@
* intentional for performance concerns. Hence, this class does not expose the contained node and
* should not be modified to do so in future when used with multiple threads. This class choose to
* achieve thread-safety through locks rather than just creating and returning immutable instances
* to its caller because the combining of a large and wide trie require per-node copy which has
* exponential cost and more expensive than synchronization.
* to its caller because the combining of a large and wide trie require per-node copy which is
* more expensive than synchronization.
*
* <p>Note: {@link #equals(Object)}, {@link #hashCode()} of this class are not synchronized and if
* their usage needs synchronization then the client should do it.
Expand Down Expand Up @@ -197,28 +198,36 @@ public synchronized void add(Iterable<String> segments) {
* @param other The other {@link BoundedTrieData} to combine with.
* @return The combined {@link BoundedTrieData}.
*/
public synchronized BoundedTrieData combine(@Nonnull BoundedTrieData other) {
if (other.root == null && other.singleton == null) {
return this;
}
public BoundedTrieData combine(@Nonnull BoundedTrieData other) {
BoundedTrieData otherDeepCopy;
// other can be modified in some different thread, and we need to atomically access
// its fields to combine correctly. Furthermore, simply doing this under synchronized(other)
// is not safe as it might lead to deadlock. Assume the current thread got lock on
// 'this' and is executing combine with `other` and waiting to get a lock on `other`
// while some other thread is performing `other.combiner(this)` and waiting to get a
// lock on `this` object.
BoundedTrieData otherDeepCopy = other.getCumulative();
if (this.root == null && this.singleton == null) {
return otherDeepCopy;
// its fields to combine correctly. Furthermore, doing this whole method under
// synchronized(other) is not safe as it might lead to deadlock. Assume the current
// thread got lock on 'this' and is executing combine with `other` and waiting to get a
// lock on `other` while some other thread is performing `other.combiner(this)` and
// waiting to get a lock on `this` object.
// Here it is safe to get a lock on other as we don't yet hold a lock on this to end up with
// race condition.
synchronized (other) {
if (other.root == null && other.singleton == null) {
return this;
}
otherDeepCopy = other.getCumulative();
}
otherDeepCopy.root = otherDeepCopy.asTrie();
otherDeepCopy.singleton = null;
otherDeepCopy.root.merge(this.asTrie());
otherDeepCopy.bound = Math.min(this.bound, otherDeepCopy.bound);
while (otherDeepCopy.root.getSize() > otherDeepCopy.bound) {
otherDeepCopy.root.trim();

synchronized (this) {
if (this.root == null && this.singleton == null) {
return otherDeepCopy;
}
otherDeepCopy.root = otherDeepCopy.asTrie();
otherDeepCopy.singleton = null;
otherDeepCopy.root.merge(this.asTrie());
otherDeepCopy.bound = Math.min(this.bound, otherDeepCopy.bound);
while (otherDeepCopy.root.getSize() > otherDeepCopy.bound) {
otherDeepCopy.root.trim();
}
return otherDeepCopy;
}
return otherDeepCopy;
}

/**
Expand Down Expand Up @@ -287,6 +296,7 @@ public final String toString() {
* of the tree itself. This avoids acquiring and release N nodes in a path. This class is not
* intended to be used directly outside of {@link BoundedTrieData} with multiple threads.
*/
@VisibleForTesting
static class BoundedTrieNode implements Serializable {

public static final String TRUNCATED_TRUE = String.valueOf(true);
Expand Down Expand Up @@ -375,6 +385,7 @@ int add(List<String> segments) {
* @param segmentsIter An iterator over the paths to add.
* @return The total change in the size of the subtree rooted at this node.
*/
@VisibleForTesting
int addAll(List<List<String>> segmentsIter) {
return segmentsIter.stream().mapToInt(this::add).sum();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -166,7 +166,7 @@ private static Iterable<MetricResult<BoundedTrieResult>> extractBoundedTrieMetri
return monitoringInfoList.stream()
.filter(item -> BOUNDED_TRIE_TYPE.equals(item.getType()))
.filter(item -> item.getLabelsMap().get(NAMESPACE_LABEL) != null)
.map(PortableMetrics::convertBoundedTrieMonitoringInfoToStringSet)
.map(PortableMetrics::convertBoundedTrieMonitoringInfoToBoundedTrie)
.collect(Collectors.toList());
}

Expand All @@ -183,7 +183,7 @@ private static MetricResult<StringSetResult> convertStringSetMonitoringInfoToStr
return MetricResult.create(key, false, result);
}

private static MetricResult<BoundedTrieResult> convertBoundedTrieMonitoringInfoToStringSet(
private static MetricResult<BoundedTrieResult> convertBoundedTrieMonitoringInfoToBoundedTrie(
MetricsApi.MonitoringInfo monitoringInfo) {
Map<String, String> labelsMap = monitoringInfo.getLabelsMap();
MetricKey key =
Expand Down

0 comments on commit db4e944

Please sign in to comment.