[ 
https://issues.apache.org/jira/browse/CASSANDRA-9382?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Yuki Morishita updated CASSANDRA-9382:
--------------------------------------
    Attachment: yjp-heapdump.png

I had a chance to look at the heap dump from the original issue, and found that 
the leak is coming from SSTable's {{readMeter}} sync task.

When SSTable is getting closed, {{readMeter}} sync task is cancelled, but it is 
only cancelled if it's not running. Since task is scheduled to run every 5 min, 
that task is scheduled again holding reference to deleted SSTable.

This is only affected in versions before 2.1.5, CASSANDRA-8707 changed to call 
[{{cancel}} with interruption when the task is 
running|https://github.com/apache/cassandra/blob/cassandra-2.1.5/src/java/org/apache/cassandra/io/sstable/SSTableReader.java#L2270].

> Snapshot file descriptors not getting purged (possible fd leak)
> ---------------------------------------------------------------
>
>                 Key: CASSANDRA-9382
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-9382
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Mark Curtis
>            Assignee: Yuki Morishita
>         Attachments: yjp-heapdump.png
>
>
> OpsCenter has the repair service which does a lot of small range repairs. 
> Each repair would generate a snapshot as per normal. The cluster was showing 
> a steady increase in disk space over the course of a couple of days and the 
> only way to workaround the issue was to restart the node.
> Upon some further inspection it was seen that a lsof output of the cassandra 
> process was still showing file descriptors for snapshots that no longer 
> existed on the file system. For example:
> {code}
> ava    5822 cassandra  DEL    REG             202,32                 7359833 
> /media/ephemeral1/cassandra/data/somekeyspace/table1/snapshots/669a3a30-f3d3-11e4-bec6-3f6c4fb06498/somekeyspace-table1-jb-897689-Data.db
> {code}
> We also took a heapdump which basically showed the same thing, lots of 
> references to these file handles. We checked the logs for any errors 
> especially relating to repairs that might have failed but there was nothing 
> observed
> The repair service logs in OpsCenter showed also that all repairs (1000s of 
> them) had completed successfully, again showing that there was no repair 
> issue.
> I have not yet been able to reproduce the issue locally on a test box. The 
> cluster that this original issue appeared on was a production cluster with 
> the following spec:
> cassandra_versions: 2.0.14.352
> cluster_cores : 8, 
> cluster_instance_types : i2.2xlarge
> cluster_os : Amazon linux amd64 
> node count: 4
> node java version: Oracle Java 1.7.0_51



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to