Optimize name-based queries to use ArrayBackedSortedColumns patch by Pavel Yaskevich; reviewed by slebresne and jbellis for CASSANDRA-5043
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/db9eb04e Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/db9eb04e Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/db9eb04e Branch: refs/heads/trunk Commit: db9eb04e2700408303222e1b4bd0096af69399b5 Parents: 31ba289 Author: Jonathan Ellis <[email protected]> Authored: Tue Dec 11 13:44:21 2012 -0600 Committer: Jonathan Ellis <[email protected]> Committed: Tue Dec 11 13:44:21 2012 -0600 ---------------------------------------------------------------------- CHANGES.txt | 1 + .../apache/cassandra/db/CollationController.java | 40 +++++++++------ 2 files changed, 25 insertions(+), 16 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/db9eb04e/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index abb7bad..2a70cec 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 1.2.1 + * Optimize name-based queries to use ArrayBackedSortedColumns (CASSANDRA-5043) * Fall back to old manifest if most recent is unparseable (CASSANDRA-5041) * pool [Compressed]RandomAccessReader objects on the partitioned read path (CASSANDRA-4942) http://git-wip-us.apache.org/repos/asf/cassandra/blob/db9eb04e/src/java/org/apache/cassandra/db/CollationController.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/CollationController.java b/src/java/org/apache/cassandra/db/CollationController.java index 7160b62..3350de3 100644 --- a/src/java/org/apache/cassandra/db/CollationController.java +++ b/src/java/org/apache/cassandra/db/CollationController.java @@ -25,7 +25,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.cassandra.db.columniterator.OnDiskAtomIterator; -import org.apache.cassandra.db.columniterator.ISSTableColumnIterator; import org.apache.cassandra.db.columniterator.SimpleAbstractColumnIterator; import org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy; import org.apache.cassandra.db.filter.NamesQueryFilter; @@ -36,14 +35,15 @@ import org.apache.cassandra.io.sstable.SSTableReader; import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.utils.CloseableIterator; +import org.apache.cassandra.utils.HeapAllocator; public class CollationController { private static final Logger logger = LoggerFactory.getLogger(CollationController.class); private final ColumnFamilyStore cfs; - private final boolean mutableColumns; private final QueryFilter filter; + private final ISortedColumns.Factory factory; private final int gcBefore; private int sstablesIterated = 0; @@ -51,9 +51,13 @@ public class CollationController public CollationController(ColumnFamilyStore cfs, boolean mutableColumns, QueryFilter filter, int gcBefore) { this.cfs = cfs; - this.mutableColumns = mutableColumns; this.filter = filter; this.gcBefore = gcBefore; + + // AtomicSortedColumns doesn't work for super columns (see #3821) + this.factory = mutableColumns + ? cfs.metadata.cfType == ColumnFamilyType.Super ? ThreadSafeSortedColumns.factory() : AtomicSortedColumns.factory() + : ArrayBackedSortedColumns.factory(); } public ColumnFamily getTopLevelColumns() @@ -73,15 +77,17 @@ public class CollationController private ColumnFamily collectTimeOrderedData() { logger.trace("collectTimeOrderedData"); - - // AtomicSortedColumns doesn't work for super columi ns (see #3821) - ISortedColumns.Factory factory = mutableColumns - ? cfs.metadata.cfType == ColumnFamilyType.Super ? ThreadSafeSortedColumns.factory() : AtomicSortedColumns.factory() - : TreeMapBackedSortedColumns.factory(); ColumnFamily container = ColumnFamily.create(cfs.metadata, factory, filter.filter.isReversed()); List<OnDiskAtomIterator> iterators = new ArrayList<OnDiskAtomIterator>(); Tracing.trace("Acquiring sstable references"); ColumnFamilyStore.ViewFragment view = cfs.markReferenced(filter.key); + + // We use a temporary CF object per memtable or sstable source so we can accomodate this.factory being ABSC, + // which requires addAtom to happen in sorted order. Then we use addAll to merge into the final collection, + // which allows a (sorted) set of columns to be merged even if they are not uniformly sorted after the existing + // ones. + ColumnFamily temp = ColumnFamily.create(cfs.metadata, ArrayBackedSortedColumns.factory(), filter.filter.isReversed()); + try { Tracing.trace("Merging memtable contents"); @@ -91,10 +97,13 @@ public class CollationController if (iter != null) { iterators.add(iter); - container.delete(iter.getColumnFamily()); + temp.delete(iter.getColumnFamily()); while (iter.hasNext()) - container.addAtom(iter.next()); + temp.addAtom(iter.next()); } + + container.addAll(temp, HeapAllocator.instance); + temp.clear(); } // avoid changing the filter columns of the original filter @@ -132,12 +141,15 @@ public class CollationController mostRecentRowTombstone = cf.deletionInfo().getTopLevelDeletion().markedForDeleteAt; } - container.delete(cf); + temp.delete(cf); sstablesIterated++; Tracing.trace("Merging data from sstable {}", sstable.descriptor.generation); while (iter.hasNext()) - container.addAtom(iter.next()); + temp.addAtom(iter.next()); } + + container.addAll(temp, HeapAllocator.instance); + temp.clear(); } // we need to distinguish between "there is no data at all for this row" (BF will let us rebuild that efficiently) @@ -220,10 +232,6 @@ public class CollationController private ColumnFamily collectAllData() { logger.trace("collectAllData"); - // AtomicSortedColumns doesn't work for super columns (see #3821) - ISortedColumns.Factory factory = mutableColumns - ? cfs.metadata.cfType == ColumnFamilyType.Super ? ThreadSafeSortedColumns.factory() : AtomicSortedColumns.factory() - : ArrayBackedSortedColumns.factory(); Tracing.trace("Acquiring sstable references"); ColumnFamilyStore.ViewFragment view = cfs.markReferenced(filter.key); List<OnDiskAtomIterator> iterators = new ArrayList<OnDiskAtomIterator>(Iterables.size(view.memtables) + view.sstables.size());
