[
https://issues.apache.org/jira/browse/CASSANDRA-6283?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13870606#comment-13870606
]
Andreas Schnitzerling commented on CASSANDRA-6283:
--------------------------------------------------
Hello,
this problem is occurred, when node was not shutdown probably. As I know, that
issue is known as CASSANDRA-6531. Here is the stack trace:
{panel:title=system.log}
ERROR [ReadStage:2385] 2014-01-14 10:57:11,875 CassandraDaemon.java (line 187)
Exception in thread Thread[ReadStage:2385,5,main]
java.lang.RuntimeException: java.lang.IllegalArgumentException: bufferSize must
be positive
at
org.apache.cassandra.service.RangeSliceVerbHandler.doVerb(RangeSliceVerbHandler.java:49)
at
org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:60)
at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
at java.lang.Thread.run(Unknown Source)
Caused by: java.lang.IllegalArgumentException: bufferSize must be positive
at
org.apache.cassandra.io.util.RandomAccessReader.<init>(RandomAccessReader.java:75)
at
org.apache.cassandra.io.compress.CompressedRandomAccessReader.<init>(CompressedRandomAccessReader.java:76)
at
org.apache.cassandra.io.compress.CompressedRandomAccessReader.open(CompressedRandomAccessReader.java:43)
at
org.apache.cassandra.io.util.CompressedPoolingSegmentedFile.createReader(CompressedPoolingSegmentedFile.java:48)
at
org.apache.cassandra.io.util.PoolingSegmentedFile.getSegment(PoolingSegmentedFile.java:39)
at
org.apache.cassandra.io.sstable.SSTableReader.getFileDataInput(SSTableReader.java:1195)
at
org.apache.cassandra.db.columniterator.SimpleSliceReader.<init>(SimpleSliceReader.java:57)
at
org.apache.cassandra.db.columniterator.SSTableSliceIterator.createReader(SSTableSliceIterator.java:65)
at
org.apache.cassandra.db.columniterator.SSTableSliceIterator.<init>(SSTableSliceIterator.java:42)
at
org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:167)
at
org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:62)
at
org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:250)
at
org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:53)
at
org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1516)
at
org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1335)
at
org.apache.cassandra.db.index.composites.CompositesSearcher$1.computeNext(CompositesSearcher.java:245)
at
org.apache.cassandra.db.index.composites.CompositesSearcher$1.computeNext(CompositesSearcher.java:105)
at
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
at
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
at
org.apache.cassandra.db.ColumnFamilyStore.filter(ColumnFamilyStore.java:1710)
at
org.apache.cassandra.db.index.composites.CompositesSearcher.search(CompositesSearcher.java:53)
at
org.apache.cassandra.db.index.SecondaryIndexManager.search(SecondaryIndexManager.java:537)
at
org.apache.cassandra.db.ColumnFamilyStore.search(ColumnFamilyStore.java:1698)
at
org.apache.cassandra.db.RangeSliceCommand.executeLocally(RangeSliceCommand.java:135)
at
org.apache.cassandra.service.RangeSliceVerbHandler.doVerb(RangeSliceVerbHandler.java:39)
... 4 more
ERROR [Finalizer] 2014-01-14 10:57:12,005 RandomAccessReader.java (line 398)
LEAK finalizer had to clean up
java.lang.Exception: RAR for
D:\Programme\cassandra\data\nieste\niesteinverters\nieste-niesteinverters-jb-2669-Data.db
allocated
at
org.apache.cassandra.io.util.RandomAccessReader.<init>(RandomAccessReader.java:66)
at
org.apache.cassandra.io.compress.CompressedRandomAccessReader.<init>(CompressedRandomAccessReader.java:76)
at
org.apache.cassandra.io.compress.CompressedRandomAccessReader.open(CompressedRandomAccessReader.java:43)
at
org.apache.cassandra.io.util.CompressedPoolingSegmentedFile.createReader(CompressedPoolingSegmentedFile.java:48)
at
org.apache.cassandra.io.util.PoolingSegmentedFile.getSegment(PoolingSegmentedFile.java:39)
at
org.apache.cassandra.io.sstable.SSTableReader.getFileDataInput(SSTableReader.java:1195)
at
org.apache.cassandra.db.columniterator.SimpleSliceReader.<init>(SimpleSliceReader.java:57)
at
org.apache.cassandra.db.columniterator.SSTableSliceIterator.createReader(SSTableSliceIterator.java:65)
at
org.apache.cassandra.db.columniterator.SSTableSliceIterator.<init>(SSTableSliceIterator.java:42)
at
org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:167)
at
org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:62)
at
org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:250)
at
org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:53)
at
org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1516)
at
org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1335)
at
org.apache.cassandra.db.index.composites.CompositesSearcher$1.computeNext(CompositesSearcher.java:245)
at
org.apache.cassandra.db.index.composites.CompositesSearcher$1.computeNext(CompositesSearcher.java:105)
at
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
at
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
at
org.apache.cassandra.db.ColumnFamilyStore.filter(ColumnFamilyStore.java:1710)
at
org.apache.cassandra.db.index.composites.CompositesSearcher.search(CompositesSearcher.java:53)
at
org.apache.cassandra.db.index.SecondaryIndexManager.search(SecondaryIndexManager.java:537)
at
org.apache.cassandra.db.ColumnFamilyStore.search(ColumnFamilyStore.java:1698)
at
org.apache.cassandra.db.RangeSliceCommand.executeLocally(RangeSliceCommand.java:135)
at
org.apache.cassandra.service.RangeSliceVerbHandler.doVerb(RangeSliceVerbHandler.java:39)
at
org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:60)
at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
at java.lang.Thread.run(Unknown Source)
ERROR [CompactionExecutor:446] 2014-01-14 11:02:57,342 CassandraDaemon.java
(line 187) Exception in thread Thread[CompactionExecutor:446,1,main]
java.lang.IllegalArgumentException: bufferSize must be positive
at
org.apache.cassandra.io.util.RandomAccessReader.<init>(RandomAccessReader.java:75)
at
org.apache.cassandra.io.compress.CompressedRandomAccessReader.<init>(CompressedRandomAccessReader.java:76)
at
org.apache.cassandra.io.compress.CompressedThrottledReader.<init>(CompressedThrottledReader.java:34)
at
org.apache.cassandra.io.compress.CompressedThrottledReader.open(CompressedThrottledReader.java:48)
at
org.apache.cassandra.io.sstable.SSTableReader.openDataReader(SSTableReader.java:1355)
at
org.apache.cassandra.io.sstable.SSTableScanner.<init>(SSTableScanner.java:67)
at
org.apache.cassandra.io.sstable.SSTableReader.getScanner(SSTableReader.java:1161)
at
org.apache.cassandra.io.sstable.SSTableReader.getScanner(SSTableReader.java:1173)
at
org.apache.cassandra.db.compaction.AbstractCompactionStrategy.getScanners(AbstractCompactionStrategy.java:244)
at
org.apache.cassandra.db.compaction.AbstractCompactionStrategy.getScanners(AbstractCompactionStrategy.java:250)
at
org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:126)
at
org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
at
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
at
org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60)
at
org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
at
org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:197)
at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
at java.util.concurrent.FutureTask$Sync.innerRun(Unknown Source)
at java.util.concurrent.FutureTask.run(Unknown Source)
at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
at java.lang.Thread.run(Unknown Source)
ERROR [Finalizer] 2014-01-14 11:02:57,552 RandomAccessReader.java (line 398)
LEAK finalizer had to clean up
java.lang.Exception: RAR for
D:\Programme\cassandra\data\KSlogdata\CFlogdata\KSlogdata-CFlogdata-jb-32763-Data.db
allocated
at
org.apache.cassandra.io.util.RandomAccessReader.<init>(RandomAccessReader.java:66)
at
org.apache.cassandra.io.compress.CompressedRandomAccessReader.<init>(CompressedRandomAccessReader.java:76)
at
org.apache.cassandra.io.compress.CompressedThrottledReader.<init>(CompressedThrottledReader.java:34)
at
org.apache.cassandra.io.compress.CompressedThrottledReader.open(CompressedThrottledReader.java:48)
at
org.apache.cassandra.io.sstable.SSTableReader.openDataReader(SSTableReader.java:1355)
at
org.apache.cassandra.io.sstable.SSTableScanner.<init>(SSTableScanner.java:67)
at
org.apache.cassandra.io.sstable.SSTableReader.getScanner(SSTableReader.java:1161)
at
org.apache.cassandra.io.sstable.SSTableReader.getScanner(SSTableReader.java:1173)
at
org.apache.cassandra.db.compaction.AbstractCompactionStrategy.getScanners(AbstractCompactionStrategy.java:244)
at
org.apache.cassandra.db.compaction.AbstractCompactionStrategy.getScanners(AbstractCompactionStrategy.java:250)
at
org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:126)
at
org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
at
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
at
org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60)
at
org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
at
org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:197)
at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
at java.util.concurrent.FutureTask$Sync.innerRun(Unknown Source)
at java.util.concurrent.FutureTask.run(Unknown Source)
at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
at java.lang.Thread.run(Unknown Source)
{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)