[ 
https://issues.apache.org/jira/browse/CASSANDRA-17205?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17459347#comment-17459347
 ] 

Josh McKenzie commented on CASSANDRA-17205:
-------------------------------------------

[branch|https://github.com/apache/cassandra/compare/trunk...josh-mckenzie:cassandra-17205?expand=1]

[JDK11 
CI|https://app.circleci.com/pipelines/github/josh-mckenzie/cassandra/151/workflows/ed12dd8c-a3d9-4b87-a895-903132502221]

 Functionally changes in a couple subtle ways:

Before, we'd assess if the {{Tracker}} existed and if the {{ColumnFamilyStore}} 
was valid *at time of tidy run*; we instead evaluate now at time of *creation* 
of the {{SSTableTidier}}, meaning the {{Tracker}} could go out of scope and be 
otherwise nullified before the two operations that relied on its presence for 
evaluation.

Those 2 operations are pretty benign however; clearing the {{SSTableReadMeter}} 
in {{system.sstable_activity}}, and decrementing the {{totalDiskSpaceUsed}} 
metric in the {{ColumnFamilyStore}}. Both operations we should be able to just 
try and gracefully fail if the Tracker's no longer active or valid by time of 
SSTableTidier run.

Second, we now grab a reference to the totalDiskSpaceUsed Counter inside the 
ColumnFamilyStore at time of {{SSTableTidier}} creation instead of at run() 
runtime, meaning we could theoretically hold a handle to a metric on a 
ColumnFamilyStore that's dropped in the interim. I don't _think_ this should be 
a problem; the code around releasing metrics in {{TableMetrics.releaseMetric}} 
should still run w/out issue and remove the metric from the top level registry, 
we'll just hold a ref to it and operate on the {{LongAdder}} in the {{Counter}} 
and then drop the ref to it when the {{SSTableTidier}} gets collected.

So in short: I don't love it and I've commented in the diff around these 
subtleties, but it sidesteps the current strong ref loop we have to the Tracker.

Going to give CI a bit to run, make sure things are clean before I find a 
reviewer.     

> File leaks will not be be detected and released due to strong self-references 
> in the tidier
> -------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-17205
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-17205
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Josh McKenzie
>            Priority: Normal
>
> LogTransaction.SSTableTidier holds a reference to a {{Tracker}} which holds 
> references to both a {{ColumnFamilyStore}} and a {{View}}, both of which hold 
> refs to SSTableReaders. As per the comment at the top of the SSTableTidier:
> {quote}// must not retain a reference to the SSTableReader, else leak 
> detection cannot kick in
> {quote}
> We shouldn't hold a reference to the Tracker here; long running unit tests 
> w/-Dcassandra.debugrefcount=true had this pop up.
> {code}ERROR [Strong-Reference-Leak-Detector:1] 2020-10-27T01:10:12,421 
> NoSpamLogger.java:97 - Strong self-ref loop detected{code}



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to