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

Jonathan Ellis commented on CASSANDRA-283:
------------------------------------------

Actually now that I think of it I did fix a FD leak in 287 already.  In the code

               iter = filter.getSSTableColumnIterator(sstable);
               if (iter.hasNext())
               {
                   returnCF.delete(iter.getColumnFamily());
                   iterators.add(iter);
               }
               else
               {
                   iter.close();
               }

the "else" clause didn't exist before, and since it wasn't added to the list of 
iterators it wouldn't get closed at the end.

> Cassandra leaks FDs
> -------------------
>
>                 Key: CASSANDRA-283
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-283
>             Project: Cassandra
>          Issue Type: Bug
>         Environment: Debian Etch, J2SE 1.6.0
>            Reporter: Ian Eure
>
> Cassandra leaks file descriptors like crazy. I started getting these errors 
> after a few hours of uptime:
> java.lang.RuntimeException: java.io.FileNotFoundException: 
> /var/cassandra/data/Digg-Items-2-Data.db (Too many open files)
>       at 
> org.apache.cassandra.service.CassandraServer.readColumnFamily(CassandraServer.java:84)
>       at 
> org.apache.cassandra.service.CassandraServer.get_slice(CassandraServer.java:181)
>       at 
> org.apache.cassandra.service.Cassandra$Processor$get_slice.process(Cassandra.java:859)
>       at 
> org.apache.cassandra.service.Cassandra$Processor.process(Cassandra.java:817)
>       at 
> org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:252)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>       at java.lang.Thread.run(Thread.java:619)
> Caused by: java.io.FileNotFoundException: 
> /var/cassandra/data/Digg-Items-2-Data.db (Too many open files)
>       at java.io.RandomAccessFile.open(Native Method)
>       at java.io.RandomAccessFile.<init>(RandomAccessFile.java:212)
>       at java.io.RandomAccessFile.<init>(RandomAccessFile.java:98)
>       at 
> org.apache.cassandra.io.BufferedRandomAccessFile.<init>(BufferedRandomAccessFile.java:141)
>       at 
> org.apache.cassandra.io.SequenceFile$BufferReader.init(SequenceFile.java:811)
>       at 
> org.apache.cassandra.io.SequenceFile$Reader.<init>(SequenceFile.java:743)
>       at 
> org.apache.cassandra.io.SequenceFile$BufferReader.<init>(SequenceFile.java:805)
>       at 
> org.apache.cassandra.io.SequenceFile$ColumnGroupReader.<init>(SequenceFile.java:248)
>       at 
> org.apache.cassandra.io.SSTableReader.getColumnGroupReader(SSTableReader.java:346)
>       at 
> org.apache.cassandra.db.SSTableColumnIterator.<init>(ColumnIterator.java:61)
>       at 
> org.apache.cassandra.db.ColumnFamilyStore.getSliceFrom(ColumnFamilyStore.java:1589)
>       at org.apache.cassandra.db.Table.getRow(Table.java:596)
>       at 
> org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:60)
>       at 
> org.apache.cassandra.service.StorageProxy.weakReadLocal(StorageProxy.java:600)
>       at 
> org.apache.cassandra.service.StorageProxy.readProtocol(StorageProxy.java:303)
>       at 
> org.apache.cassandra.service.CassandraServer.readColumnFamily(CassandraServer.java:80)
> I have an open file limit of 1024. Examining the lsof output for Cassandra 
> shows 975 FDs for the same file: /var/cassandra/data/Digg-Items-2-Data.db
> Clearly, these FDs are leaking somewhere.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.

Reply via email to