[
https://issues.apache.org/jira/browse/CASSANDRA-7872?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14159459#comment-14159459
]
Oleg Anastasyev commented on CASSANDRA-7872:
--------------------------------------------
No, Phattom ref proc does not require STW FullGC.
On Java 6, 7, and 8 it gets processed during CMS remark phase (if you enable
PrintGCDetails you'll see timing of this stage in logs, like for example:
{code}
[Rescan (parallel) , 0.0159280 secs]
[weak refs processing, 0.0032600 secs] <--- here
[class unloading, 0.0082150 secs][scrub symbol table, 0.0078220 secs][scrub
string table, 0.0013500 secs] [1 CMS-remark: 4480383K(24117248K)]
4506007K(28835840K), 0.1030490 secs] [Times: user=1.00 sys=0.01, real=0.10
secs]
{code}
This is how it works since 1.6_24 and all more modern JDKs, including all java
7 and on. jdk 6_24 is 3 years old now, so this way of work can be considered
stable.
Could you provide details on which java or java build number it did not work
this way ?
> ensure compacted obsolete sstables are not open on node restart and nodetool
> refresh, even on sstable reference miscounting or deletion tasks are failed.
> ---------------------------------------------------------------------------------------------------------------------------------------------------------
>
> Key: CASSANDRA-7872
> URL: https://issues.apache.org/jira/browse/CASSANDRA-7872
> Project: Cassandra
> Issue Type: Bug
> Components: Core
> Reporter: Oleg Anastasyev
> Assignee: Oleg Anastasyev
> Fix For: 2.0.11
>
> Attachments: EnsureNoObsoleteSSTables-7872-v2.0.txt
>
>
> Since CASSANDRA-4436 compacted sstables are no more marked with
> COMPACTED_MARKER file. Instead after they are compacted, DataTracker calls
> SSTableReader.markObsolete(), but the actual deletion is happening later on
> SSTableReader.releaseReference().
> This reference counting is very fragile, it is very easy to introduce a
> hard-to-catch and rare bug, so this reference count never reaches 0 ( like
> CASSANDRA-6503 for example )
> This means, that very rarely obsolete sstable files are not removed from disk
> (but are not used anymore by cassandra to read data).
> If more than gc grace time has passed since sstable file was not removed from
> disk and operator issues either nodetool refresh or just reboots a node,
> these obsolete files are being discovered and open for read by a node. So
> deleted data is resurrected, being quickly spread by RR to whole cluster.
> Because consequences are very serious (even a single not removed obsolete
> sstable file could render your data useless) this patch makes sure no
> obsolete sstable file can be open for read by:
> 1. Removing sstables on CFS init analyzing sstable generations (sstable is
> removed, if there are another sstable, listing this as ancestor)
> 2. Reimplementing COMPACTED_MARKER file for sstable. This marker is created
> as soon as markObsolete is called. This is neccessary b/c generation info can
> be lost (when sstables compact to none)
> 3. To remove sstables sooner then restart - reimplemented the good old GC
> phantom reference queue as well.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)