Author: jbellis
Date: Wed Feb 17 16:59:18 2010
New Revision: 911074

URL: http://svn.apache.org/viewvc?rev=911074&view=rev
Log:
use while loop instead of recursion when trimming sstables compaction list to 
avoid blowing stack in pathological cases.
patch by jbellis; reviewed by gdusbabe for CASSANDRA-804

Modified:
    
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/CompactionManager.java

Modified: 
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/CompactionManager.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/CompactionManager.java?rev=911074&r1=911073&r2=911074&view=diff
==============================================================================
--- 
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/CompactionManager.java
 (original)
+++ 
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/CompactionManager.java
 Wed Feb 17 16:59:18 2010
@@ -247,13 +247,19 @@
         String compactionFileLocation = 
table.getDataFileLocation(cfs.getExpectedCompactedFileSize(sstables));
         // If the compaction file path is null that means we have no space 
left for this compaction.
         // try again w/o the largest one.
+        List<SSTableReader> smallerSSTables = new 
ArrayList<SSTableReader>(sstables);
+        while (compactionFileLocation == null && smallerSSTables.size() > 1)
+        {
+            logger.warn("insufficient space to compact all requested files " + 
StringUtils.join(smallerSSTables, ", "));
+            smallerSSTables.remove(cfs.getMaxSizeFile(smallerSSTables));
+            compactionFileLocation = 
table.getDataFileLocation(cfs.getExpectedCompactedFileSize(smallerSSTables));
+        }
         if (compactionFileLocation == null)
         {
-            SSTableReader maxFile = cfs.getMaxSizeFile(sstables);
-            List<SSTableReader> smallerSSTables = new 
ArrayList<SSTableReader>(sstables);
-            smallerSSTables.remove(maxFile);
-            return doCompaction(cfs, smallerSSTables, gcBefore);
+            logger.error("insufficient space to compact even the two smallest 
files, aborting");
+            return 0;
         }
+        sstables = smallerSSTables;
 
         // new sstables from flush can be added during a compaction, but only 
the compaction can remove them,
         // so in our single-threaded compaction world this is a valid way of 
determining if we're compacting


Reply via email to