Repository: cassandra Updated Branches: refs/heads/cassandra-2.0 7216639e5 -> c2563dbc8
Hint streaming can cause decommission to fail patch by jasobrown, reviewed by marcuse for CASSANDRA-7219 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/2f497eda Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2f497eda Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2f497eda Branch: refs/heads/cassandra-2.0 Commit: 2f497eda0f121fce68256e18748d04bd7cbfe032 Parents: ea26dbd Author: Jason Brown <jasobr...@apple.com> Authored: Fri May 30 05:40:30 2014 -0700 Committer: Jason Brown <jasobr...@apple.com> Committed: Fri May 30 05:40:30 2014 -0700 ---------------------------------------------------------------------- CHANGES.txt | 1 + src/java/org/apache/cassandra/service/StorageService.java | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/2f497eda/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index ba3a675..ef5f72a 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -26,6 +26,7 @@ * exit CQLSH with error status code if script fails (CASSANDRA-6344) * Fix bug with some IN queries missig results (CASSANDRA-7105) * Fix availability validation for LOCAL_ONE CL (CASSANDRA-7319) + * Hint streaming can cause decommission to fail (CASSANDRA-7219) 1.2.16 http://git-wip-us.apache.org/repos/asf/cassandra/blob/2f497eda/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 8e7c073..494da33 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -2966,7 +2966,8 @@ public class StorageService extends NotificationBroadcasterSupport implements IE private CountDownLatch streamHints() { - if (HintedHandOffManager.instance.listEndpointsPendingHints().size() == 0) + ColumnFamilyStore hintsCF = Table.open(Table.SYSTEM_KS).getColumnFamilyStore(SystemTable.HINTS_CF); + if (hintsCF.getMemtableColumnsCount() == 0 && hintsCF.estimateKeys() == 0) return new CountDownLatch(0); // gather all live nodes in the cluster that aren't also leaving