Skip to content

Commit

Permalink
switched from isPresent to maybePresent. Dont call getBloomFilter and…
Browse files Browse the repository at this point in the history
… directly access bf to avoid resource leak detection
  • Loading branch information
dcapwell committed Apr 20, 2022
1 parent a9e3811 commit 3aa3373
Show file tree
Hide file tree
Showing 3 changed files with 4 additions and 6 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,6 @@
import org.apache.cassandra.io.util.FileDataInput;
import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.schema.CompactionParams.TombstoneOption;
import org.apache.cassandra.utils.AlwaysPresentFilter;
import org.apache.cassandra.utils.OverlapIterator;
import org.apache.cassandra.utils.concurrent.Refs;

Expand Down Expand Up @@ -255,7 +254,7 @@ public LongPredicate getPurgeEvaluator(DecoratedKey key)

for (SSTableReader sstable: filteredSSTables)
{
if (sstable.isPresent(key))
if (sstable.maybePresent(key))
{
minTimestampSeen = Math.min(minTimestampSeen, sstable.getMinTimestamp());
hasTimestamp = true;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -998,7 +998,7 @@ private static Collection<SSTableReader> sstablesInBounds(ColumnFamilyStore cfs,

public void forceCompactionForKey(ColumnFamilyStore cfStore, DecoratedKey key)
{
forceCompaction(cfStore, () -> sstablesWithKey(cfStore, key), sstable -> sstable.isPresent(key));
forceCompaction(cfStore, () -> sstablesWithKey(cfStore, key), sstable -> sstable.maybePresent(key));
}

private static Collection<SSTableReader> sstablesWithKey(ColumnFamilyStore cfs, DecoratedKey key)
Expand All @@ -1008,7 +1008,7 @@ private static Collection<SSTableReader> sstablesWithKey(ColumnFamilyStore cfs,
key.getToken().maxKeyBound());
for (SSTableReader sstable : liveTables)
{
if (sstable.isPresent(key))
if (sstable.maybePresent(key))
sstables.add(sstable);
}
return sstables.isEmpty() ? Collections.emptyList() : sstables;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2003,11 +2003,10 @@ public void addTo(Ref.IdentityCollection identities)

}

public boolean isPresent(DecoratedKey key)
public boolean maybePresent(DecoratedKey key)
{
// if we don't have bloom filter(bf_fp_chance=1.0 or filter file is missing),
// we check index file instead.
IFilter bf = getBloomFilter();
return bf instanceof AlwaysPresentFilter && getPosition(key, Operator.EQ, false) != null || bf.isPresent(key);
}

Expand Down

0 comments on commit 3aa3373

Please sign in to comment.