avoid generating redundant compaction tasks during streaming patch by yukim; reviewed by jbellis for CASSANDRA-4174
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/150b3167 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/150b3167 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/150b3167 Branch: refs/heads/trunk Commit: 150b3167f2f9952a4537e0d3c754d094a9a097b2 Parents: 5b85f3c Author: Jonathan Ellis <[email protected]> Authored: Tue Apr 24 13:07:34 2012 -0500 Committer: Jonathan Ellis <[email protected]> Committed: Tue Apr 24 13:11:47 2012 -0500 ---------------------------------------------------------------------- CHANGES.txt | 2 ++ .../org/apache/cassandra/db/ColumnFamilyStore.java | 7 ++++++- .../cassandra/streaming/StreamInSession.java | 6 ++++-- 3 files changed, 12 insertions(+), 3 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/150b3167/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index f91a375..47f95ce 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,6 @@ 1.1.1-dev + * avoid generating redundant compaction tasks during streaming + (CASSANDRA-4174) * add -cf option to nodetool snapshot, and takeColumnFamilySnapshot to StorageService mbean (CASSANDRA-556) * optimize cleanup to drop entire sstables where possible (CASSANDRA-4079) http://git-wip-us.apache.org/repos/asf/cassandra/blob/150b3167/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 77f4dab..659be73 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java @@ -880,7 +880,12 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean public void addSSTable(SSTableReader sstable) { assert sstable.getColumnFamilyName().equals(columnFamily); - data.addSSTables(Arrays.asList(sstable)); + addSSTables(Arrays.asList(sstable)); + } + + public void addSSTables(Collection<SSTableReader> sstables) + { + data.addSSTables(sstables); CompactionManager.instance.submitBackground(this); } http://git-wip-us.apache.org/repos/asf/cassandra/blob/150b3167/src/java/org/apache/cassandra/streaming/StreamInSession.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/streaming/StreamInSession.java b/src/java/org/apache/cassandra/streaming/StreamInSession.java index a5e08f0..1660aaa 100644 --- a/src/java/org/apache/cassandra/streaming/StreamInSession.java +++ b/src/java/org/apache/cassandra/streaming/StreamInSession.java @@ -165,17 +165,19 @@ public class StreamInSession extends AbstractStreamSession throw new AssertionError("We shouldn't fail acquiring a reference on a sstable that has just been transferred"); ColumnFamilyStore cfs = Table.open(sstable.getTableName()).getColumnFamilyStore(sstable.getColumnFamilyName()); - cfs.addSSTable(sstable); if (!cfstores.containsKey(cfs)) cfstores.put(cfs, new ArrayList<SSTableReader>()); cfstores.get(cfs).add(sstable); } - // build secondary indexes + // add sstables and build secondary indexes for (Map.Entry<ColumnFamilyStore, List<SSTableReader>> entry : cfstores.entrySet()) { if (entry.getKey() != null) + { + entry.getKey().addSSTables(entry.getValue()); entry.getKey().indexManager.maybeBuildSecondaryIndexes(entry.getValue(), entry.getKey().indexManager.getIndexedColumns()); + } } } finally
