Skip to content

Commit

Permalink
no need to store the hashSum for view. Calculate on remove vs add
Browse files Browse the repository at this point in the history
  • Loading branch information
Yukei7 committed Dec 24, 2024
1 parent f126b06 commit fac664b
Show file tree
Hide file tree
Showing 2 changed files with 10 additions and 8 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -59,9 +59,7 @@ public static SSTableIntervalTree build(Iterable<SSTableReader> sstables)

public static SSTableIntervalTree replaceSSTables(SSTableIntervalTree tree, Iterable<SSTableReader> sstables)
{
List<Interval<PartitionPosition, SSTableReader>> intervals = buildIntervals(sstables);
SSTableIntervalTree newTree = new SSTableIntervalTree(tree.count, tree.head.copyAndReplace(intervals));
return newTree;
return new SSTableIntervalTree(tree.count, tree.head.copyAndReplace(buildIntervals(sstables)));
}

public static List<Interval<PartitionPosition, SSTableReader>> buildIntervals(Iterable<SSTableReader> sstables)
Expand Down
14 changes: 9 additions & 5 deletions src/java/org/apache/cassandra/db/lifecycle/View.java
Original file line number Diff line number Diff line change
Expand Up @@ -307,16 +307,15 @@ static Function<View, View> updateLiveSet(final Set<SSTableReader> remove, final
public View apply(View view)
{
Map<SSTableReader, SSTableReader> sstableMap = replace(view.sstablesMap, remove, add);
int afterHashSum = getSSTablesHash(sstableMap.keySet());
if (view.sstablesHash == afterHashSum)
if (isRangesUnchanged(remove, add))
{
// If the SSTable before and after has the same key range, then there is no need to rebuild the interval tree.
// Instead, we find and update the node
return new View(view.liveMemtables, view.flushingMemtables, sstableMap, view.compactingMap,
SSTableIntervalTree.replaceSSTables(view.intervalTree, add), afterHashSum);
SSTableIntervalTree.replaceSSTables(view.intervalTree, add));
}
return new View(view.liveMemtables, view.flushingMemtables, sstableMap, view.compactingMap,
SSTableIntervalTree.build(sstableMap.keySet()), afterHashSum);
SSTableIntervalTree.build(sstableMap.keySet()));
}
};
}
Expand Down Expand Up @@ -386,7 +385,7 @@ public boolean apply(T t)
};
}

private static int getSSTablesHash(Collection<SSTableReader> readers) {
private static int getSSTablesHash(Iterable<SSTableReader> readers) {
int hashSum = 0;
for (SSTableReader reader : readers) {
if (reader != null) {
Expand All @@ -395,4 +394,9 @@ private static int getSSTablesHash(Collection<SSTableReader> readers) {
}
return hashSum;
}

private static boolean isRangesUnchanged(final Set<SSTableReader> remove, final Iterable<SSTableReader> add)
{
return getSSTablesHash(remove) == getSSTablesHash(add);
}
}

0 comments on commit fac664b

Please sign in to comment.