always compact away deleted hints immediately after handoff
patch by brandonwilliams and jbellis for CASSANDRA-3955


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/5cc12f37
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/5cc12f37
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/5cc12f37

Branch: refs/heads/cassandra-1.1
Commit: 5cc12f37f6419a80efaa7341e8a284e19165ef30
Parents: e2ac530
Author: Jonathan Ellis <jbel...@apache.org>
Authored: Fri Feb 24 10:35:58 2012 -0600
Committer: Jonathan Ellis <jbel...@apache.org>
Committed: Fri Feb 24 10:35:58 2012 -0600

----------------------------------------------------------------------
 CHANGES.txt                                        |    4 ++
 .../apache/cassandra/db/HintedHandOffManager.java  |   23 +++++++++-----
 2 files changed, 19 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5cc12f37/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 9cb9414..31ae77d 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,3 +1,7 @@
+1.0.9
+ * always compact away deleted hints immediately after handoff (CASSANDRA-3955)
+
+
 1.0.8
  * fix race between cleanup and flush on secondary index CFSes (CASSANDRA-3712)
  * avoid including non-queried nodes in rangeslice read repair

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5cc12f37/src/java/org/apache/cassandra/db/HintedHandOffManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/HintedHandOffManager.java 
b/src/java/org/apache/cassandra/db/HintedHandOffManager.java
index e2dc046..ca416d7 100644
--- a/src/java/org/apache/cassandra/db/HintedHandOffManager.java
+++ b/src/java/org/apache/cassandra/db/HintedHandOffManager.java
@@ -25,13 +25,13 @@ import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
 import java.util.*;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
+import java.util.concurrent.*;
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
 
 import com.google.common.collect.ImmutableSortedSet;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -166,7 +166,6 @@ public class HintedHandOffManager implements 
HintedHandOffManagerMBean
             return;
         Token<?> token = 
StorageService.instance.getTokenMetadata().getToken(endpoint);
         ByteBuffer tokenBytes = 
StorageService.getPartitioner().getTokenFactory().toByteArray(token);
-        final ColumnFamilyStore hintStore = 
Table.open(Table.SYSTEM_TABLE).getColumnFamilyStore(HINTS_CF);
         final RowMutation rm = new RowMutation(Table.SYSTEM_TABLE, tokenBytes);
         rm.delete(new QueryPath(HINTS_CF), System.currentTimeMillis());
 
@@ -179,8 +178,7 @@ public class HintedHandOffManager implements 
HintedHandOffManagerMBean
                 {
                     logger_.info("Deleting any stored hints for " + endpoint);
                     rm.apply();
-                    hintStore.forceBlockingFlush();
-                    CompactionManager.instance.submitMaximal(hintStore, 
Integer.MAX_VALUE);
+                    compact();
                 }
                 catch (Exception e)
                 {
@@ -191,6 +189,16 @@ public class HintedHandOffManager implements 
HintedHandOffManagerMBean
         StorageService.optionalTasks.execute(runnable);
     }
 
+    private Future<?> compact() throws ExecutionException, InterruptedException
+    {
+        final ColumnFamilyStore hintStore = 
Table.open(Table.SYSTEM_TABLE).getColumnFamilyStore(HINTS_CF);
+        hintStore.forceBlockingFlush();
+        ArrayList<Descriptor> descriptors = new ArrayList<Descriptor>();
+        for (SSTable sstable : hintStore.getSSTables())
+            descriptors.add(sstable.descriptor);
+        return CompactionManager.instance.submitUserDefined(hintStore, 
descriptors, Integer.MAX_VALUE);
+    }
+
     private static boolean pagingFinished(ColumnFamily hintColumnFamily, 
ByteBuffer startColumn)
     {
         // done if no hints found or the start column (same as last column 
processed in previous iteration) is the only one
@@ -361,8 +369,7 @@ public class HintedHandOffManager implements 
HintedHandOffManagerMBean
         {
             try
             {
-                hintStore.forceBlockingFlush();
-                CompactionManager.instance.submitMaximal(hintStore, 
Integer.MAX_VALUE).get();
+                compact().get();
             }
             catch (Exception e)
             {

Reply via email to