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

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

Regarding the LEAK on repair -par:  graham sanderson on the 17th appeared to be 
on the mark.  Adding FileUtils.closeQuietly(validator) in the finally block on 
StreamWriter.java plugged all LEAK messages from nodetool.bat repair -par.  
I've started looking at the other leaks Andreas linked on here and the 1st I 
tracked down doesn't look like it should be leaking:
{code:title=LEAK message}
ERROR [Finalizer] 2014-02-18 12:53:42,388 RandomAccessReader.java (line 394) 
LEAK finalizer had to clean up 
java.lang.Exception: RAR for 
D:\Programme\cassandra\data\system\schema_keyspaces\system-schema_keyspaces-jb-433-Data.db
 allocated
at 
org.apache.cassandra.io.util.RandomAccessReader.<init>(RandomAccessReader.java:63)
at 
org.apache.cassandra.io.compress.CompressedRandomAccessReader.<init>(CompressedRandomAccessReader.java:76)
at 
org.apache.cassandra.io.compress.CompressedRandomAccessReader.open(CompressedRandomAccessReader.java:55)
at 
org.apache.cassandra.io.sstable.SSTableReader.openDataReader(SSTableReader.java:1362)
at org.apache.cassandra.io.sstable.SSTableScanner.<init>(SSTableScanner.java:67)
at 
org.apache.cassandra.io.sstable.SSTableReader.getScanner(SSTableReader.java:1147)
at 
org.apache.cassandra.db.RowIteratorFactory.getIterator(RowIteratorFactory.java:69)
at 
org.apache.cassandra.db.ColumnFamilyStore.getSequentialIterator(ColumnFamilyStore.java:1599)
at 
org.apache.cassandra.db.ColumnFamilyStore.getRangeSlice(ColumnFamilyStore.java:1718)
at 
org.apache.cassandra.db.ColumnFamilyStore.getRangeSlice(ColumnFamilyStore.java:1656)
at 
org.apache.cassandra.db.SystemKeyspace.serializedSchema(SystemKeyspace.java:767)
at org.apache.cassandra.db.DefsTables.loadFromKeyspace(DefsTables.java:121)
at 
org.apache.cassandra.config.DatabaseDescriptor.loadSchemas(DatabaseDescriptor.java:525)
at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:242)
at 
org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:462)
at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:552)
{code}
Which should be closed at:
{code:title=ColumnFamilyStore.filter()}
...
        finally
        {
            try
            {
                rowIterator.close();
                Tracing.trace("Scanned {} rows and matched {}", total, matched);
            }
...
{code}

I'll look into the LEAK messages on some of the others you got Andreas and see 
if the code where the stack was captured looks like it leaks the handle.

Regarding the locked file handles on snapshots: It's not looking great for our 
options on hard links for the snashots on non-par repair.  On win7 I basically 
can't touch hard links to locked files which is different than the note 
mentioned on StackExchange:

{code:title=win7 move / delete}
C:\Users\jmckenzie\Desktop\FileLocker>mklink/H link.txt test.txt
Hardlink created for link.txt <<===>> test.txt
C:\Users\jmckenzie\Desktop\FileLocker>rename link.txt newlink.txt
The process cannot access the file because it is being used by another process.
C:\Users\jmckenzie\Desktop\FileLocker>move link.txt newlink.txt
The process cannot access the file because it is being used by another process.
        0 file(s) moved.
{code}

And as for behavior on winxp - it looks like this behavior was the same there 
as far as not being able to delete hard links to locked files:
{code:title=winxp delete hard link}
C:\Documents and Settings\jmckenzie\Desktop>fsutil hardlink create link.txt 
test.txt
Hardlink created for C:\Documents and Settings\jmckenzie\Desktop\link.txt <<===>
> C:\Documents and Settings\jmckenzie\Desktop\test.txt
C:\Documents and Settings\jmckenzie\Desktop>del link.txt
C:\Documents and Settings\jmckenzie\Desktop\link.txt
The process cannot access the file because it is being used by another process.
{code}

I believe this is why we have SSTableDeletingTask.java and the recurrent runs 
on CMS GC and rescheduling self.  We could follow the implementation pattern 
and have a SnapshotFileDeletingTask and register w/StorageService.tasks the 
same way we do on SSTableDeleting, but that doesn't solve the locked file 
handle move problems we have unfortunately.  It doesn't look like we use 
FileUtils.renameWithConfirm() in too many places in our code-base so it might 
be ok to document that as unsafe on Windows and deal with the other case.

> 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)

Reply via email to