Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
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
7 changes: 7 additions & 0 deletions src/java/org/apache/cassandra/utils/concurrent/Ref.java
Original file line number Diff line number Diff line change
Expand Up @@ -130,6 +130,13 @@ public Ref(T referent, Tidy tidy)
public void release()
{
state.release(false);
// We require a reachability fence here to prevent the JIT from clearing the Ref reference from the stack
// during the state.release() call. If that happens, the ref may become phantom reachable, and the GC may clear
// the state as a phantom reference to the ref and then enqueue the state on the phantom queue. This allows
// the Reference-Reaper to race with our state.release call above, and the reference-reaper may be able to
// update the released flag before the non-leak release path. In this case, we report a spurious leak and a
// spurious bad release.
Reference.reachabilityFence(this);
}

public Throwable ensureReleased(Throwable accumulate)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
import org.apache.cassandra.io.util.File;
import org.apache.cassandra.utils.concurrent.Ref;
import org.jboss.byteman.contrib.bmunit.BMRule;
import org.jboss.byteman.contrib.bmunit.BMRules;
import org.jboss.byteman.contrib.bmunit.BMUnitRunner;

import static org.apache.cassandra.config.CassandraRelevantProperties.TEST_DEBUG_REF_COUNT;
Expand All @@ -60,6 +61,12 @@ public static void setup()
System.setProperty("org.jboss.byteman.verbose", "true"); // checkstyle: suppress nearby 'blockSystemPropertyUsage'
}

public static void maybeGc()
{
if (ThreadLocalRandom.current().nextDouble() < 0.001d)
System.gc();
}

/**
* This is a basic concurrency test for {@link SSTableReader.GlobalTidy}
*
Expand All @@ -72,11 +79,17 @@ public static void setup()
* - GlobalTidy.lookup modification when the relevant Ref is alive
*/
@Test
@BMRule(name = "Count leaks",
targetClass="Ref$State",
targetMethod="reportLeak",
targetLocation="AT EXIT",
action="org.apache.cassandra.io.sstable.format.GlobalTidyConcurrencyTest.leakHappened = true;")
@BMRules(rules = {
@BMRule(name = "Count leaks",
targetClass="Ref$State",
targetMethod="reportLeak",
targetLocation="AT EXIT",
action="org.apache.cassandra.io.sstable.format.GlobalTidyConcurrencyTest.leakHappened = true;"),
@BMRule(name = "Trigger occasional GCs to stress reference reaper",
targetClass="Ref$State",
targetMethod="release",
targetLocation="AT ENTRY",
action="org.apache.cassandra.io.sstable.format.GlobalTidyConcurrencyTest.maybeGc();")})
public void tidyVsGetRaceTest() throws InterruptedException
{
int NUM_THREADS = 32;
Expand Down