Author: slebresne
Date: Mon Nov 28 14:58:53 2011
New Revision: 1207262

URL: http://svn.apache.org/viewvc?rev=1207262&view=rev
Log:
Shutdown CL after having flushed non-durable CF
patch by slebresne; reviewed by jbellis for CASSANDRA-3520

Modified:
    cassandra/branches/cassandra-0.8/CHANGES.txt
    
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/service/StorageService.java

Modified: cassandra/branches/cassandra-0.8/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/CHANGES.txt?rev=1207262&r1=1207261&r2=1207262&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.8/CHANGES.txt (original)
+++ cassandra/branches/cassandra-0.8/CHANGES.txt Mon Nov 28 14:58:53 2011
@@ -42,6 +42,7 @@
  * Fix bug preventing the use of efficient cross-DC writes (CASSANDRA-3472)
  * (Hadoop) skip empty rows when entire row is requested, redux 
(CASSANDRA-2855)
  * fix concurrence issue in the FailureDetector (CASSANDRA-3519)
+ * shutdown CL after having flushed non-durable CF (CASSANDRA-3520)
 
 
 0.8.7

Modified: 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/service/StorageService.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/service/StorageService.java?rev=1207262&r1=1207261&r2=1207262&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/service/StorageService.java
 (original)
+++ 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/service/StorageService.java
 Mon Nov 28 14:58:53 2011
@@ -404,12 +404,11 @@ public class StorageService implements I
             public void runMayThrow() throws ExecutionException, 
InterruptedException, IOException
             {
                 ThreadPoolExecutor mutationStage = 
StageManager.getStage(Stage.MUTATION);
-                if (!mutationStage.isShutdown())
-                {
-                    mutationStage.shutdown();
-                    mutationStage.awaitTermination(1, TimeUnit.SECONDS);
-                    CommitLog.instance.shutdownBlocking();
-                }
+                if (mutationStage.isShutdown())
+                    return; // drained already
+
+                mutationStage.shutdown();
+                mutationStage.awaitTermination(1, TimeUnit.SECONDS);
 
                 List<Future<?>> flushes = new ArrayList<Future<?>>();
                 for (Table table : Table.all())
@@ -426,6 +425,8 @@ public class StorageService implements I
                     }
                 }
                 FBUtilities.waitOnFutures(flushes);
+
+                CommitLog.instance.shutdownBlocking();
             }
         });
         Runtime.getRuntime().addShutdownHook(drainOnShutdown);


Reply via email to