cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Joshua McKenzie (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-6283) Windows 7 data files keept open / can't be deleted after compaction.
Date Mon, 17 Feb 2014 16:10:21 GMT

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

Joshua McKenzie commented on CASSANDRA-6283:
--------------------------------------------

I've reproduced leaked file handles on repair in a lab on 2.0.5 w/leakfinalizer.patch.  Nodes
start up without issue - I'm not seeing any LEAK or File-Not-Found on regular init, and the
LEAK aren't showing up until repair kicks off.  W/the finalizer patch repair runs through
to completion.  Andreas - have you had a chance to try out 2.0.5 w/the patch yet?

lastly - the leaks I'm seeing look like they're all isolated to a single case - streaming
data outbound during the repair process:
ERROR [Finalizer] 2014-02-17 09:21:52,922 RandomAccessReader.java (line 399) LEAK finalizer
had to clean up
java.lang.Exception: RAR for C:\var\lib\cassandra\data\Keyspace1\Standard1\Keyspace1-Standard1-jb-41-CRC.db
allocated
        at org.apache.cassandra.io.util.RandomAccessReader.<init>(RandomAccessReader.java:66)
        at org.apache.cassandra.io.util.RandomAccessReader.open(RandomAccessReader.java:106)
        at org.apache.cassandra.io.util.RandomAccessReader.open(RandomAccessReader.java:98)
        at org.apache.cassandra.io.util.DataIntegrityMetadata$ChecksumValidator.<init>(DataIntegrityMetadata.java:53)
        at org.apache.cassandra.io.util.DataIntegrityMetadata.checksumValidator(DataIntegrityMetadata.java:40)
        at org.apache.cassandra.streaming.StreamWriter.write(StreamWriter.java:76)
        at org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:59)
        at org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:42)
        at org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:45)
        at org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.sendMessage(ConnectionHandler.java:383)
        at org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.run(ConnectionHandler.java:355)
        at java.lang.Thread.run(Thread.java:744)

Andreas - could you confirm whether or not this matches what you're seeing in your environment?
 I'm curious if this is a dangling file handle like we've seen in other related tickets or
if this is perhaps a race condition on access Windows is intolerant of.

> Windows 7 data files keept open / can't be deleted after compaction.
> --------------------------------------------------------------------
>
>                 Key: CASSANDRA-6283
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-6283
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>         Environment: Windows 7 (32) / Java 1.7.0.45
>            Reporter: Andreas Schnitzerling
>            Assignee: Joshua McKenzie
>              Labels: compaction
>             Fix For: 2.0.6
>
>         Attachments: leakdetect.patch, screenshot-1.jpg, system.log
>
>
> Files cannot be deleted, patch CASSANDRA-5383 (Win7 deleting problem) doesn't help on
Win-7 on Cassandra 2.0.2. Even 2.1 Snapshot is not running. The cause is: Opened file handles
seem to be lost and not closed properly. Win 7 blames, that another process is still using
the file (but its obviously cassandra). Only restart of the server makes the files deleted.
But after heavy using (changes) of tables, there are about 24K files in the data folder (instead
of 35 after every restart) and Cassandra crashes. I experiminted and I found out, that a finalizer
fixes the problem. So after GC the files will be deleted (not optimal, but working fine).
It runs now 2 days continously without problem. Possible fix/test:
> I wrote the following finalizer at the end of class org.apache.cassandra.io.util.RandomAccessReader:
> {code:title=RandomAccessReader.java|borderStyle=solid}
> @Override
> protected void finalize() throws Throwable {
> 	deallocate();
> 	super.finalize();
> }
> {code}
> Can somebody test / develop / patch it? Thx.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)

Mime
View raw message