Catch memtable flush exceptions during shutdown patch by Sergio Bossa; reviewed by jbellis for CASSANDRA-6735
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/b6849375 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/b6849375 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/b6849375 Branch: refs/heads/trunk Commit: b6849375151f79d0aa708a990dc9daef567dfcb5 Parents: 5e40a3b Author: Jonathan Ellis <jbel...@apache.org> Authored: Wed Feb 19 16:14:32 2014 -0600 Committer: Jonathan Ellis <jbel...@apache.org> Committed: Wed Feb 19 16:14:32 2014 -0600 ---------------------------------------------------------------------- CHANGES.txt | 1 + src/java/org/apache/cassandra/service/StorageService.java | 10 +++++++++- 2 files changed, 10 insertions(+), 1 deletion(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/b6849375/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index 51dec14..11fcf18 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 1.2.16 + * Catch memtable flush exceptions during shutdown (CASSANDRA-6735) * Don't attempt cross-dc forwarding in mixed-version cluster with 1.1 (CASSANDRA-6732) * Fix broken streams when replacing with same IP (CASSANDRA-6622) http://git-wip-us.apache.org/repos/asf/cassandra/blob/b6849375/src/java/org/apache/cassandra/service/StorageService.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index ec8c7c3..6611ba9 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -566,7 +566,15 @@ public class StorageService extends NotificationBroadcasterSupport implements IE flushes.add(cfs.forceFlush()); } } - FBUtilities.waitOnFutures(flushes); + try + { + FBUtilities.waitOnFutures(flushes); + } + catch (Throwable e) + { + // don't let this stop us from shutting down the commitlog and other thread pools + logger.warn("Caught exception while waiting for memtable flushes during shutdown hook", e); + } CommitLog.instance.shutdownBlocking();