fix possible stackoverflow when compacting 1000s of sstables patch by jbellis; reviewed by yukim for CASSANDRA-4765
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/e400d0ec Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e400d0ec Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e400d0ec Branch: refs/heads/cassandra-1.1 Commit: e400d0eca27f3c00e36e3871a30ac5c97891abbe Parents: c9b249b Author: Jonathan Ellis <jbel...@apache.org> Authored: Thu Oct 18 15:17:29 2012 -0500 Committer: Jonathan Ellis <jbel...@apache.org> Committed: Thu Oct 18 15:17:29 2012 -0500 ---------------------------------------------------------------------- CHANGES.txt | 2 ++ .../org/apache/cassandra/db/ColumnFamilyStore.java | 4 +++- 2 files changed, 5 insertions(+), 1 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/e400d0ec/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index 36b4758..49164b6 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,6 @@ 1.1.7 + * fix possible stackoverflow when compacting 1000s of sstables + (CASSANDRA-4765) * fix wrong leveled compaction progress calculation (CASSANDRA-4807) * add a close() method to CRAR to prevent leaking file descriptors (CASSANDRA-4820) http://git-wip-us.apache.org/repos/asf/cassandra/blob/e400d0ec/src/java/org/apache/cassandra/db/ColumnFamilyStore.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java index 64b2402..67a883d 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java @@ -876,6 +876,8 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean */ public Set<SSTableReader> getOverlappingSSTables(Collection<SSTableReader> sstables) { + logger.debug("Checking for sstables overlapping {}", sstables); + // a normal compaction won't ever have an empty sstables list, but we create a skeleton // compaction controller for streaming, and that passes an empty list. if (sstables.isEmpty()) @@ -888,7 +890,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean { Set<SSTableReader> overlaps = ImmutableSet.copyOf(tree.search(new Interval<SSTableReader>(sstable.first, sstable.last))); assert overlaps.contains(sstable); - results = results == null ? overlaps : Sets.union(results, overlaps); + results = results == null ? overlaps : Sets.union(results, overlaps).immutableCopy(); } results = Sets.difference(results, ImmutableSet.copyOf(sstables));