cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jbel...@apache.org
Subject svn commit: r1097530 - in /cassandra/branches/cassandra-0.8: CHANGES.txt src/java/org/apache/cassandra/db/CompactionManager.java
Date Thu, 28 Apr 2011 16:16:40 GMT
Author: jbellis
Date: Thu Apr 28 16:16:40 2011
New Revision: 1097530

URL: http://svn.apache.org/viewvc?rev=1097530&view=rev
Log:
make forceUserDefinedCompaction always try to attempt what is requested
patch by jbellis; reviewed by slebresne for CASSANDRA-2575

Modified:
    cassandra/branches/cassandra-0.8/CHANGES.txt
    cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/CompactionManager.java

Modified: cassandra/branches/cassandra-0.8/CHANGES.txt
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/CHANGES.txt?rev=1097530&r1=1097529&r2=1097530&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.8/CHANGES.txt (original)
+++ cassandra/branches/cassandra-0.8/CHANGES.txt Thu Apr 28 16:16:40 2011
@@ -11,6 +11,9 @@
  * fix incorrect use of NBHM.size in ReadCallback that could cause
    reads to time out even when responses were received (CASSAMDRA-2552)
  * trigger read repair correctly for LOCAL_QUORUM reads (CASSANDRA-2556)
+ * forceUserDefinedCompaction will attempt to compact what it is given
+   even if the pessimistic estimate is that there is not enough disk space;
+   automatic compactions will only compact 2 or more sstables (CASSANDRA-2575)
 
 
 0.8.0-beta1

Modified: cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/CompactionManager.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/CompactionManager.java?rev=1097530&r1=1097529&r2=1097530&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/CompactionManager.java
(original)
+++ cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/CompactionManager.java
Thu Apr 28 16:16:40 2011
@@ -414,10 +414,11 @@ public class CompactionManager implement
                     // attempt to schedule the set
                     else if ((sstables = cfs.getDataTracker().markCompacting(sstables, 1,
Integer.MAX_VALUE)) != null)
                     {
+                        String location = cfs.table.getDataFileLocation(1);
                         // success: perform the compaction
                         try
                         {
-                            doCompaction(cfs, sstables, gcBefore);
+                            doCompactionWithoutSizeEstimation(cfs, sstables, gcBefore, location);
                         }
                         finally
                         {
@@ -489,11 +490,32 @@ public class CompactionManager implement
         }
     }
 
+    int doCompaction(ColumnFamilyStore cfs, Collection<SSTableReader> sstables, int
gcBefore) throws IOException
+    {
+        Table table = cfs.table;
+
+        // If the compaction file path is null that means we have no space left for this
compaction.
+        // try again w/o the largest one.
+        Set<SSTableReader> smallerSSTables = new HashSet<SSTableReader>(sstables);
+        while (smallerSSTables.size() > 1)
+        {
+            String compactionFileLocation = table.getDataFileLocation(cfs.getExpectedCompactedFileSize(smallerSSTables));
+            if (compactionFileLocation != null)
+                return doCompactionWithoutSizeEstimation(cfs, smallerSSTables, gcBefore,
compactionFileLocation);
+
+            logger.warn("insufficient space to compact all requested files " + StringUtils.join(smallerSSTables,
", "));
+            smallerSSTables.remove(cfs.getMaxSizeFile(smallerSSTables));
+        }
+
+        logger.error("insufficient space to compact even the two smallest files, aborting");
+        return 0;
+    }
+
     /**
      * For internal use and testing only.  The rest of the system should go through the submit*
methods,
      * which are properly serialized.
      */
-    int doCompaction(ColumnFamilyStore cfs, Collection<SSTableReader> sstables, int
gcBefore) throws IOException
+    int doCompactionWithoutSizeEstimation(ColumnFamilyStore cfs, Collection<SSTableReader>
sstables, int gcBefore, String compactionFileLocation) throws IOException
     {
         // The collection of sstables passed may be empty (but not null); even if
         // it is not empty, it may compact down to nothing if all rows are deleted.
@@ -507,23 +529,6 @@ public class CompactionManager implement
         for (SSTableReader sstable : sstables)
             assert sstable.descriptor.cfname.equals(cfs.columnFamily);
 
-        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)
-        {
-            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
         // all the sstables (that existed when we started)



Mime
View raw message