Need to close files in loadBloomFilter and loadIndexFile
--------------------------------------------------------

                 Key: CASSANDRA-533
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-533
             Project: Cassandra
          Issue Type: Bug
          Components: Core
    Affects Versions: 0.4
         Environment: Vista, Cassandra 0.4.1
            Reporter: Tim Freeman


When starting Cassandra on a Windows system, I intermittently see errors like:

DEBUG - Expected bloom filter size : 2560
DEBUG - collecting Generation:false:4...@9
DEBUG - collecting Token:false:1...@0
INFO - Saved Token found: 25027551081353517716727338628156823602
ERROR - Error in ThreadPoolExecutor
java.util.concurrent.ExecutionException: java.io.IOException: Failed to delete 
LocationInfo-8-Index.db
        at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222)
        at java.util.concurrent.FutureTask.get(FutureTask.java:83)
        at 
org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.afterExecute(DebuggableThreadPoolExecutor.java:8
6)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:888)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
        at java.lang.Thread.run(Thread.java:619)
Caused by: java.io.IOException: Failed to delete LocationInfo-8-Index.db
        at 
org.apache.cassandra.utils.FileUtils.deleteWithConfirm(FileUtils.java:55)
        at org.apache.cassandra.io.SSTableReader.delete(SSTableReader.java:269)
        at 
org.apache.cassandra.db.ColumnFamilyStore.doFileCompaction(ColumnFamilyStore.java:1145)
        at 
org.apache.cassandra.db.ColumnFamilyStore.doCompaction(ColumnFamilyStore.java:686)
        at 
org.apache.cassandra.db.MinorCompactionManager$1.call(MinorCompactionManager.java:166)
        at 
org.apache.cassandra.db.MinorCompactionManager$1.call(MinorCompactionManager.java:163)
        at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
        at java.util.concurrent.FutureTask.run(FutureTask.java:138)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
        ... 2 more
DEBUG - Starting to listen on 127.0.0.1:7001
DEBUG - Binding thrift service to localhost:9160

The problem here is that the files are not closed immediately after they are 
read, and on Windows you can't delete an open file.  In general, failing to 
close a file causes more subtle problems on a Unix system.  You can run out of 
file descriptors if files are opened faster than the garbage collector closes 
them, and you can run out of disk space if the total data in the 
opened-but-deleted files is enough to fill the disk.

I see two places to fix, SSTableReader.loadIndexFile and 
SSTableReader.loadBloomFilter.  Here are revised definitions for those two 
methods:

    private void loadBloomFilter() throws IOException
    {
        DataInputStream stream = new DataInputStream(new 
FileInputStream(filterFilename()));
        try {
                bf = BloomFilter.serializer().deserialize(stream);
        } finally {
                stream.close();
        }
    }

    private void loadIndexFile() throws IOException
    {
        BufferedRandomAccessFile input = new 
BufferedRandomAccessFile(indexFilename(), "r");
        try {
                indexPositions = new ArrayList<KeyPosition>();

                int i = 0;
                long indexSize = input.length();
                while (true)
                {
                        long indexPosition = input.getFilePointer();
                        if (indexPosition == indexSize)
                        {
                                break;
                        }
                        String decoratedKey = input.readUTF();
                        input.readLong();
                        if (i++ % INDEX_INTERVAL == 0)
                        {
                                indexPositions.add(new 
KeyPosition(decoratedKey, indexPosition));
                        }
                }
        } finally {
                input.close();
        }
    }   

I have not yet tested those changes, but they look desirable even if they don't 
fix the symptom I'm experiencing.  I did not search the code for other places 
that files are opened but not closed.


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