[ https://issues.apache.org/jira/browse/CASSANDRA-6283?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13870952#comment-13870952 ]
Andreas Schnitzerling edited comment on CASSANDRA-6283 at 1/14/14 5:52 PM: --------------------------------------------------------------------------- Yesterday I updated one node with 2.0.4-rel incl. finalizer-patch (see results above). Nodetool repair -par caused node to repair "endless" and collecting about 65K Files in datafolder. I updated now to pre-2.0.5 from today (commit f6f50ddffe0821617fe29482f9ec918608560381). After starting, a lot of LEAK messages and File-Not-Found messages appeared in system.log. But files reduce. {panel:title=system.log (pre-2.0.5)} ERROR [SSTableBatchOpen:1] 2014-01-14 18:18:42,753 CassandraDaemon.java:139 - Exception in thread Thread[SSTableBatchOpen:1,5,main] java.lang.RuntimeException: java.io.FileNotFoundException: D:\Programme\cassandra\data\KSlogdata\CFlogdata\KSlogdata-CFlogdata-jb-27051-Index.db (Das System kann die angegebene Datei nicht finden) at org.apache.cassandra.io.util.RandomAccessReader.open(RandomAccessReader.java:109) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.util.RandomAccessReader.open(RandomAccessReader.java:97) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.sstable.SSTableReader.buildSummary(SSTableReader.java:595) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:575) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:527) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:328) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:230) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.sstable.SSTableReader$4.run(SSTableReader.java:364) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source) ~[na:1.7.0_25] at java.util.concurrent.FutureTask$Sync.innerRun(Unknown Source) ~[na:1.7.0_25] at java.util.concurrent.FutureTask.run(Unknown Source) ~[na:1.7.0_25] at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source) ~[na:1.7.0_25] at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source) ~[na:1.7.0_25] at java.lang.Thread.run(Unknown Source) ~[na:1.7.0_25] Caused by: java.io.FileNotFoundException: D:\Programme\cassandra\data\KSlogdata\CFlogdata\KSlogdata-CFlogdata-jb-27051-Index.db (Das System kann die angegebene Datei nicht finden) at java.io.RandomAccessFile.open(Native Method) ~[na:1.7.0_25] at java.io.RandomAccessFile.<init>(Unknown Source) ~[na:1.7.0_25] at org.apache.cassandra.io.util.RandomAccessReader.<init>(RandomAccessReader.java:63) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.util.RandomAccessReader.open(RandomAccessReader.java:105) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] ... 13 common frames omitted ... ERROR [Finalizer] 2014-01-14 18:27:45,076 RandomAccessReader.java:401 - LEAK finalizer had to clean up java.lang.Exception: RAR for D:\Programme\cassandra\data\system\compactions_in_progress\system-compactions_in_progress-ka-5012-Statistics.db allocated at org.apache.cassandra.io.util.RandomAccessReader.<init>(RandomAccessReader.java:65) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.util.RandomAccessReader.open(RandomAccessReader.java:105) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.util.RandomAccessReader.open(RandomAccessReader.java:97) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:88) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:98) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.sstable.SSTableReader.getApproximateKeyCount(SSTableReader.java:167) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:125) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:66) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:198) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source) ~[na:1.7.0_25] at java.util.concurrent.FutureTask$Sync.innerRun(Unknown Source) ~[na:1.7.0_25] at java.util.concurrent.FutureTask.run(Unknown Source) ~[na:1.7.0_25] at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source) ~[na:1.7.0_25] at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source) ~[na:1.7.0_25] at java.lang.Thread.run(Unknown Source) ~[na:1.7.0_25] ... ERROR [main] 2014-01-14 18:27:45,446 CassandraDaemon.java:435 - Exception encountered during startup java.lang.NullPointerException: null at org.apache.cassandra.db.Directories.<init>(Directories.java:192) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.db.ColumnFamilyStore.removeUnfinishedCompactionLeftovers(ColumnFamilyStore.java:487) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:211) [apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:418) [apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:505) [apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] >>Cassandra shutted down<< {panel} I deleted files in folder "\data\system\compactions_in_progress". Now its not shutting down anymore. Still leak failures, no "FileNotFound" yet. Now it is compacting (automatic) no reducing, still 40K files. was (Author: andie78): Yesterday I Updated one node with 2.0.4-rel incl. Finalizer-Patch (see results above). nodetool repair -par caused node to repair "endless" and collecting about 65K Files in datafolder. I updated now to pre-2.0.5 from today (commit f6f50ddffe0821617fe29482f9ec918608560381). After starting, a lot of LEAK messages and File-Not-Found messages. But Files reduce. {panel:title=system.log (pre-2.0.5)} ERROR [SSTableBatchOpen:1] 2014-01-14 18:18:42,753 CassandraDaemon.java:139 - Exception in thread Thread[SSTableBatchOpen:1,5,main] java.lang.RuntimeException: java.io.FileNotFoundException: D:\Programme\cassandra\data\KSlogdata\CFlogdata\KSlogdata-CFlogdata-jb-27051-Index.db (Das System kann die angegebene Datei nicht finden) at org.apache.cassandra.io.util.RandomAccessReader.open(RandomAccessReader.java:109) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.util.RandomAccessReader.open(RandomAccessReader.java:97) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.sstable.SSTableReader.buildSummary(SSTableReader.java:595) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:575) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:527) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:328) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:230) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.sstable.SSTableReader$4.run(SSTableReader.java:364) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source) ~[na:1.7.0_25] at java.util.concurrent.FutureTask$Sync.innerRun(Unknown Source) ~[na:1.7.0_25] at java.util.concurrent.FutureTask.run(Unknown Source) ~[na:1.7.0_25] at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source) ~[na:1.7.0_25] at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source) ~[na:1.7.0_25] at java.lang.Thread.run(Unknown Source) ~[na:1.7.0_25] Caused by: java.io.FileNotFoundException: D:\Programme\cassandra\data\KSlogdata\CFlogdata\KSlogdata-CFlogdata-jb-27051-Index.db (Das System kann die angegebene Datei nicht finden) at java.io.RandomAccessFile.open(Native Method) ~[na:1.7.0_25] at java.io.RandomAccessFile.<init>(Unknown Source) ~[na:1.7.0_25] at org.apache.cassandra.io.util.RandomAccessReader.<init>(RandomAccessReader.java:63) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.util.RandomAccessReader.open(RandomAccessReader.java:105) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] ... 13 common frames omitted ... ERROR [Finalizer] 2014-01-14 18:27:45,076 RandomAccessReader.java:401 - LEAK finalizer had to clean up java.lang.Exception: RAR for D:\Programme\cassandra\data\system\compactions_in_progress\system-compactions_in_progress-ka-5012-Statistics.db allocated at org.apache.cassandra.io.util.RandomAccessReader.<init>(RandomAccessReader.java:65) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.util.RandomAccessReader.open(RandomAccessReader.java:105) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.util.RandomAccessReader.open(RandomAccessReader.java:97) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:88) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:98) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.sstable.SSTableReader.getApproximateKeyCount(SSTableReader.java:167) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:125) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:66) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:198) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source) ~[na:1.7.0_25] at java.util.concurrent.FutureTask$Sync.innerRun(Unknown Source) ~[na:1.7.0_25] at java.util.concurrent.FutureTask.run(Unknown Source) ~[na:1.7.0_25] at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source) ~[na:1.7.0_25] at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source) ~[na:1.7.0_25] at java.lang.Thread.run(Unknown Source) ~[na:1.7.0_25] ... ERROR [main] 2014-01-14 18:27:45,446 CassandraDaemon.java:435 - Exception encountered during startup java.lang.NullPointerException: null at org.apache.cassandra.db.Directories.<init>(Directories.java:192) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.db.ColumnFamilyStore.removeUnfinishedCompactionLeftovers(ColumnFamilyStore.java:487) ~[apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:211) [apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:418) [apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:505) [apache-cassandra-2.1-SNAPSHOT.jar:2.1-SNAPSHOT] >>Cassandra shutted down<< {panel} > 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 > Labels: compaction > Fix For: 2.0.5 > > 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)