Updated Branches: refs/heads/cassandra-1.0 5aad64a69 -> 5bcfcbc3e refs/heads/cassandra-1.1 b1c60d2b3 -> efba67888 refs/heads/trunk 44f4cac62 -> 4f1e5e288
merge from 1.1 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/4f1e5e28 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/4f1e5e28 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/4f1e5e28 Branch: refs/heads/trunk Commit: 4f1e5e2886ec13953069b579c71cc00a0dd0de8f Parents: 44f4cac efba678 Author: Jonathan Ellis <[email protected]> Authored: Mon Jul 2 03:05:47 2012 -0500 Committer: Jonathan Ellis <[email protected]> Committed: Mon Jul 2 03:09:59 2012 -0500 ---------------------------------------------------------------------- CHANGES.txt | 5 +++++ .../cassandra/db/AbstractColumnContainer.java | 4 ++-- .../cassandra/db/ArrayBackedSortedColumns.java | 1 - src/java/org/apache/cassandra/db/Column.java | 2 +- src/java/org/apache/cassandra/db/DeletionInfo.java | 4 +++- src/java/org/apache/cassandra/db/IColumn.java | 10 ++++++++-- .../org/apache/cassandra/db/IColumnContainer.java | 2 +- .../db/compaction/CompactionIterable.java | 10 +++------- .../cassandra/db/compaction/PrecompactedRow.java | 2 +- 9 files changed, 24 insertions(+), 16 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/4f1e5e28/CHANGES.txt ---------------------------------------------------------------------- diff --cc CHANGES.txt index 1955476,72991f1..7f36e9f --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,33 -1,10 +1,38 @@@ +1.2-dev + * add UseCondCardMark XX jvm settings on jdk 1.7 (CASSANDRA-4366) + * split up rpc timeout by operation type (CASSANDRA-2819) + * rewrite key cache save/load to use only sequential i/o (CASSANDRA-3762) + * update MS protocol with a version handshake + broadcast address id + (CASSANDRA-4311) + * multithreaded hint replay (CASSANDRA-4189) + * add inter-node message compression (CASSANDRA-3127) + * remove COPP (CASSANDRA-2479) + * Track tombstone expiration and compact when tombstone content is + higher than a configurable threshold, default 20% (CASSANDRA-3442) + * update MurmurHash to version 3 (CASSANDRA-2975) + * (CLI) track elapsed time for `delete' operation (CASSANDRA-4060) + * (CLI) jline version is bumped to 1.0 to properly support + 'delete' key function (CASSANDRA-4132) + * Save IndexSummary into new SSTable 'Summary' component (CASSANDRA-2392, 4289) + * Add support for range tombstones (CASSANDRA-3708) + * Improve MessagingService efficiency (CASSANDRA-3617) + * Avoid ID conflicts from concurrent schema changes (CASSANDRA-3794) + * Set thrift HSHA server thread limit to unlimited by default (CASSANDRA-4277) + * Avoids double serialization of CF id in RowMutation messages + (CASSANDRA-4293) + * stream compressed sstables directly with java nio (CASSANDRA-4297) + * Support multiple ranges in SliceQueryFilter (CASSANDRA-3885) + * Add column metadata to system column families (CASSANDRA-4018) + * (cql3) always use composite types by default (CASSANDRA-4329) + + 1.1.3 + * restore pre-CASSANDRA-3862 approach to removing expired tombstones + from row cache during compaction (CASSANDRA-4364) * (stress) support for CQL prepared statements (CASSANDRA-3633) + Merged from 1.0: + * allow dropping columns shadowed by not-yet-expired supercolumn or row + tombstones in PrecompactedRow (CASSANDRA-4396) 1.1.2 http://git-wip-us.apache.org/repos/asf/cassandra/blob/4f1e5e28/src/java/org/apache/cassandra/db/AbstractColumnContainer.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/db/AbstractColumnContainer.java index 62ed353,c7922b1..cca928b --- a/src/java/org/apache/cassandra/db/AbstractColumnContainer.java +++ b/src/java/org/apache/cassandra/db/AbstractColumnContainer.java @@@ -183,23 -180,28 +183,23 @@@ public abstract class AbstractColumnCon return columns.iterator(); } - public Iterator<IColumn> reverseIterator() - { - return columns.reverseIterator(); - } - - public Iterator<IColumn> iterator(ByteBuffer start) + public Iterator<IColumn> iterator(ColumnSlice[] slices) { - return columns.iterator(start); + return columns.iterator(slices); } - public Iterator<IColumn> reverseIterator(ByteBuffer start) + public Iterator<IColumn> reverseIterator(ColumnSlice[] slices) { - return columns.reverseIterator(start); + return columns.reverseIterator(slices); } - public boolean hasExpiredTombstones(int gcBefore) + public boolean hasIrrelevantData(int gcBefore) { - if (getLocalDeletionTime() < gcBefore) + if (deletionInfo().purge(gcBefore) == DeletionInfo.LIVE) return true; for (IColumn column : columns) - if (column.hasExpiredTombstones(gcBefore)) - if (column.mostRecentLiveChangeAt() < getLocalDeletionTime() || column.hasIrrelevantData(gcBefore)) ++ if (column.mostRecentLiveChangeAt() < deletionInfo().maxTimestamp() || column.hasIrrelevantData(gcBefore)) return true; return false; http://git-wip-us.apache.org/repos/asf/cassandra/blob/4f1e5e28/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java index 3bbf49b,246b133..4dc1e3e --- a/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java +++ b/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java @@@ -301,73 -295,58 +301,72 @@@ public class ArrayBackedSortedColumns e return new ColumnNamesSet(); } - @Override public Iterator<IColumn> iterator() { - return reversed ? reverseInternalIterator(size()) : columns.iterator(); + return reversed ? Lists.reverse(columns).iterator() : columns.iterator(); } - public Iterator<IColumn> reverseIterator() + public Iterator<IColumn> iterator(ColumnSlice[] slices) { - return reversed ? columns.iterator() : reverseInternalIterator(size()); + return new SlicesIterator(columns, comparator, slices, reversed); } - public Iterator<IColumn> iterator(ByteBuffer start) + public Iterator<IColumn> reverseIterator(ColumnSlice[] slices) { - int idx = binarySearch(start); - if (idx < 0) - idx = -idx - 1; - else if (reversed) - // listIterator.previous() doesn't return the current element at first but the previous one - idx++; - return reversed ? reverseInternalIterator(idx) : columns.listIterator(idx); + return new SlicesIterator(columns, comparator, slices, !reversed); } - public Iterator<IColumn> reverseIterator(ByteBuffer start) + private static class SlicesIterator extends AbstractIterator<IColumn> { - int idx = binarySearch(start); - if (idx < 0) - idx = -idx - 1; - else if (!reversed) - // listIterator.previous() doesn't return the current element at first but the previous one - idx++; - return reversed ? columns.listIterator(idx) : reverseInternalIterator(idx); - } + private final List<IColumn> list; + private final ColumnSlice[] slices; + private final Comparator<ByteBuffer> comparator; - private Iterator<IColumn> reverseInternalIterator(int idx) - { - final ListIterator<IColumn> iter = columns.listIterator(idx); - return new Iterator<IColumn>() + private int idx = 0; + private int previousSliceEnd = 0; + private Iterator<IColumn> currentSlice; + + public SlicesIterator(List<IColumn> list, AbstractType<?> comparator, ColumnSlice[] slices, boolean reversed) { - public boolean hasNext() - { - return iter.hasPrevious(); - } + this.list = reversed ? Lists.reverse(list) : list; + this.slices = slices; + this.comparator = reversed ? comparator.reverseComparator : comparator; + } - public IColumn next() + protected IColumn computeNext() + { + if (currentSlice == null) { - return iter.previous(); + if (idx >= slices.length) + return endOfData(); + + ColumnSlice slice = slices[idx++]; + // The first idx to include + int startIdx = slice.start.remaining() == 0 ? 0 : binarySearch(list, comparator, slice.start, previousSliceEnd); + if (startIdx < 0) + startIdx = -startIdx - 1; + + // The first idx to exclude + int finishIdx = slice.finish.remaining() == 0 ? list.size() - 1 : binarySearch(list, comparator, slice.finish, previousSliceEnd); + if (finishIdx >= 0) + finishIdx++; + else + finishIdx = -finishIdx - 1; + + if (startIdx == 0 && finishIdx == list.size()) + currentSlice = list.iterator(); + else + currentSlice = list.subList(startIdx, finishIdx).iterator(); + + previousSliceEnd = finishIdx > 0 ? finishIdx - 1 : 0; } - public void remove() - { - iter.remove(); - } - }; + if (currentSlice.hasNext()) + return currentSlice.next(); + + currentSlice = null; + return computeNext(); + } } private class ReverseSortedCollection extends AbstractCollection<IColumn> http://git-wip-us.apache.org/repos/asf/cassandra/blob/4f1e5e28/src/java/org/apache/cassandra/db/Column.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/4f1e5e28/src/java/org/apache/cassandra/db/DeletionInfo.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/db/DeletionInfo.java index 97fd4d6,0000000..5fbaec6 mode 100644,000000..100644 --- a/src/java/org/apache/cassandra/db/DeletionInfo.java +++ b/src/java/org/apache/cassandra/db/DeletionInfo.java @@@ -1,353 -1,0 +1,355 @@@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.db; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.nio.ByteBuffer; +import java.util.*; + +import com.google.common.base.Objects; +import com.google.common.collect.Iterables; ++import com.google.common.collect.Ordering; ++import com.google.common.primitives.Longs; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.io.ISerializer; +import org.apache.cassandra.io.ISSTableSerializer; +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.IntervalTree; + +public class DeletionInfo +{ + private static final Serializer serializer = new Serializer(); + + // We don't have way to represent the full interval of keys (Interval don't support the minimum token as the right bound), + // so we keep the topLevel deletion info separatly. This also slightly optimize the case of full row deletion which is rather common. + private final DeletionTime topLevel; + private final IntervalTree<ByteBuffer, DeletionTime, RangeTombstone> ranges; + + public static final DeletionInfo LIVE = new DeletionInfo(DeletionTime.LIVE, IntervalTree.<ByteBuffer, DeletionTime, RangeTombstone>emptyTree()); + + public DeletionInfo(long markedForDeleteAt, int localDeletionTime) + { + // Pre-1.1 node may return MIN_VALUE for non-deleted container, but the new default is MAX_VALUE + // (see CASSANDRA-3872) + this(new DeletionTime(markedForDeleteAt, localDeletionTime == Integer.MIN_VALUE ? Integer.MAX_VALUE : localDeletionTime), + IntervalTree.<ByteBuffer, DeletionTime, RangeTombstone>emptyTree()); + } + + public DeletionInfo(ByteBuffer start, ByteBuffer end, Comparator<ByteBuffer> comparator, long markedForDeleteAt, int localDeletionTime) + { + this(new RangeTombstone(start, end, new DeletionTime(markedForDeleteAt, localDeletionTime)), comparator); + } + + public DeletionInfo(RangeTombstone rangeTombstone, Comparator<ByteBuffer> comparator) + { + this(DeletionTime.LIVE, IntervalTree.build(Collections.<RangeTombstone>singletonList(rangeTombstone), comparator)); + assert comparator != null; + } + + private DeletionInfo(DeletionTime topLevel, IntervalTree<ByteBuffer, DeletionTime, RangeTombstone> ranges) + { + this.topLevel = topLevel; + this.ranges = ranges; + } + + public static Serializer serializer() + { + return serializer; + } + + /** + * Returns whether this DeletionInfo is live, that is deletes no columns. + */ + public boolean isLive() + { + return topLevel.markedForDeleteAt == Long.MIN_VALUE + && topLevel.localDeletionTime == Integer.MAX_VALUE + && ranges.isEmpty(); + } + + /** + * Return whether a given column is deleted by the container having this + * deletion info. + * - * @param col the column to check. ++ * @param column the column to check. + * @return true if the column is deleted, false otherwise + */ + public boolean isDeleted(IColumn column) + { + return isDeleted(column.name(), column.mostRecentLiveChangeAt()); + } + + public boolean isDeleted(ByteBuffer name, long timestamp) + { + if (isLive()) + return false; + if (timestamp <= topLevel.markedForDeleteAt) + return true; + + for (DeletionTime d : ranges.search(name)) + { + if (timestamp <= d.markedForDeleteAt) + return true; + } + return false; + } + + /** + * Return a new DeletionInfo correspond to purging every tombstones that + * are older than {@code gcbefore}. + * + * @param gcBefore timestamp (in seconds) before which tombstones should + * be purged + * @return a new DeletionInfo with the purged info remove. Should return + * DeletionInfo.LIVE if no tombstones remain. + */ + public DeletionInfo purge(int gcBefore) + { + if (ranges.isEmpty()) + { + return topLevel.localDeletionTime < gcBefore ? LIVE : this; + } + else + { + // We rebuild a new intervalTree that contains only non expired range tombstones + List<RangeTombstone> nonExpired = new ArrayList<RangeTombstone>(); + for (RangeTombstone range : ranges) + { + if (range.data.localDeletionTime >= gcBefore) + nonExpired.add(range); + } + IntervalTree<ByteBuffer, DeletionTime, RangeTombstone> newRanges = nonExpired.size() == ranges.intervalCount() + ? ranges + : IntervalTree.build(nonExpired, ranges.comparator()); + return topLevel.localDeletionTime < gcBefore + ? new DeletionInfo(DeletionTime.LIVE, newRanges) + : new DeletionInfo(topLevel, newRanges); + } + } + + /** + * Returns a new DeletionInfo containing of this plus the provided {@code + * newInfo}. + */ + public DeletionInfo add(DeletionInfo newInfo) + { + if (ranges.isEmpty()) + { + return topLevel.markedForDeleteAt < newInfo.topLevel.markedForDeleteAt + ? newInfo + : newInfo.ranges.isEmpty() ? this : new DeletionInfo(topLevel, newInfo.ranges); + } + else + { + if (newInfo.ranges.isEmpty()) + { + return topLevel.markedForDeleteAt < newInfo.topLevel.markedForDeleteAt + ? new DeletionInfo(newInfo.topLevel, ranges) + : this; + } + else + { + // Need to merge both ranges + Set<RangeTombstone> merged = new HashSet<RangeTombstone>(); + Iterables.addAll(merged, Iterables.concat(ranges, newInfo.ranges)); + return new DeletionInfo(topLevel.markedForDeleteAt < newInfo.topLevel.markedForDeleteAt ? newInfo.topLevel : topLevel, + IntervalTree.build(merged, ranges.comparator())); + } + } + } + + /** + * The maximum timestamp mentioned by this DeletionInfo. + */ + public long maxTimestamp() + { + long maxTimestamp = topLevel.markedForDeleteAt; + for (RangeTombstone i : ranges) + { + maxTimestamp = Math.max(maxTimestamp, i.data.markedForDeleteAt); + } + return maxTimestamp; + } + + public DeletionTime getTopLevelDeletion() + { + return topLevel; + } + + public Iterator<RangeTombstone> rangeIterator() + { + return ranges.iterator(); + } + + public int dataSize() + { + int size = TypeSizes.NATIVE.sizeof(topLevel.markedForDeleteAt); + for (RangeTombstone r : ranges) + { + size += r.min.remaining() + r.max.remaining(); + size += TypeSizes.NATIVE.sizeof(r.data.markedForDeleteAt); + } + return size; + } + + @Override + public String toString() + { + if (ranges.isEmpty()) + return String.format("{%s}", topLevel); + else + return String.format("{%s, ranges=%s}", topLevel, rangesAsString()); + } + + private String rangesAsString() + { + assert !ranges.isEmpty(); + StringBuilder sb = new StringBuilder(); + AbstractType at = (AbstractType)ranges.comparator(); + assert at != null; + for (RangeTombstone i : ranges) + { + sb.append("["); + sb.append(at.getString(i.min)).append("-"); + sb.append(at.getString(i.max)).append(", "); + sb.append(i.data); + sb.append("]"); + } + return sb.toString(); + } + + @Override + public boolean equals(Object o) + { + if(!(o instanceof DeletionInfo)) + return false; + DeletionInfo that = (DeletionInfo)o; + return topLevel.equals(that.topLevel) && ranges.equals(that.ranges); + } + + @Override + public final int hashCode() + { + return Objects.hashCode(topLevel, ranges); + } + + public static class Serializer implements IVersionedSerializer<DeletionInfo>, ISSTableSerializer<DeletionInfo> + { + private final static ISerializer<ByteBuffer> bbSerializer = new ISerializer<ByteBuffer>() + { + public void serialize(ByteBuffer bb, DataOutput dos) throws IOException + { + ByteBufferUtil.writeWithShortLength(bb, dos); + } + + public ByteBuffer deserialize(DataInput dis) throws IOException + { + return ByteBufferUtil.readWithShortLength(dis); + } + + public long serializedSize(ByteBuffer bb, TypeSizes typeSizes) + { + int bbSize = bb.remaining(); + return typeSizes.sizeof((short)bbSize) + bbSize; + } + }; + + private final static IntervalTree.Serializer<ByteBuffer, DeletionTime, RangeTombstone> itSerializer; + static + { + try + { + Constructor<RangeTombstone> constructor = RangeTombstone.class.getConstructor(ByteBuffer.class, ByteBuffer.class, DeletionTime.class); + itSerializer = IntervalTree.serializer(bbSerializer, DeletionTime.serializer, constructor); + } + catch (NoSuchMethodException e) + { + throw new RuntimeException(e); + } + } + + public void serialize(DeletionInfo info, DataOutput out, int version) throws IOException + { + DeletionTime.serializer.serialize(info.topLevel, out); + // Pre-1.2 version don't know about range tombstones and thus users should upgrade all + // nodes before using them. If they didn't, better fail early that propagating bad info + if (version < MessagingService.VERSION_12) + { + if (!info.ranges.isEmpty()) + throw new RuntimeException("Cannot send range tombstone to pre-1.2 node. You should upgrade all node to Cassandra 1.2+ before using range tombstone."); + // Otherwise we're done + } + else + { + itSerializer.serialize(info.ranges, out, version); + } + } + + public void serializeForSSTable(DeletionInfo info, DataOutput out) throws IOException + { + DeletionTime.serializer.serialize(info.topLevel, out); + } + + /* + * Range tombstones internally depend on the column family serializer, but it is not serialized. + * Thus deserialize(DataInput, int, Comparator<ByteBuffer>) should be used instead of this method. + */ + public DeletionInfo deserialize(DataInput in, int version) throws IOException + { + throw new UnsupportedOperationException(); + } + + public DeletionInfo deserialize(DataInput in, int version, Comparator<ByteBuffer> comparator) throws IOException + { + assert comparator != null; + DeletionTime topLevel = DeletionTime.serializer.deserialize(in); + if (version < MessagingService.VERSION_12) + return new DeletionInfo(topLevel, IntervalTree.<ByteBuffer, DeletionTime, RangeTombstone>emptyTree()); + + IntervalTree<ByteBuffer, DeletionTime, RangeTombstone> ranges = itSerializer.deserialize(in, version, comparator); + return new DeletionInfo(topLevel, ranges); + } + + public DeletionInfo deserializeFromSSTable(DataInput in, Descriptor.Version version) throws IOException + { + DeletionTime topLevel = DeletionTime.serializer.deserialize(in); + return new DeletionInfo(topLevel, IntervalTree.<ByteBuffer, DeletionTime, RangeTombstone>emptyTree()); + } + + public long serializedSize(DeletionInfo info, TypeSizes typeSizes, int version) + { + long size = DeletionTime.serializer.serializedSize(info.topLevel, typeSizes); + if (version < MessagingService.VERSION_12) + return size; + + return size + itSerializer.serializedSize(info.ranges, typeSizes, version); + } + + public long serializedSize(DeletionInfo info, int version) + { + return serializedSize(info, TypeSizes.NATIVE, version); + } + } +} http://git-wip-us.apache.org/repos/asf/cassandra/blob/4f1e5e28/src/java/org/apache/cassandra/db/IColumn.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/db/IColumn.java index 998d20f,889ad0c..0a7fe7a --- a/src/java/org/apache/cassandra/db/IColumn.java +++ b/src/java/org/apache/cassandra/db/IColumn.java @@@ -73,7 -78,13 +73,13 @@@ public interface IColumn extends OnDisk boolean isLive(); /** -- * @return true if the column or any its subcolumns expired before @param gcBefore - */ - public boolean hasIrrelevantData(int gcBefore); - - /** + * For a standard column, this is the same as timestamp(). + * For a super column, this is the max column timestamp of the sub columns. */ - public boolean hasExpiredTombstones(int gcBefore); + public long maxTimestamp(); ++ ++ /** ++ * @return true if the column or any its subcolumns is no longer relevant after @param gcBefore ++ */ ++ public boolean hasIrrelevantData(int gcBefore); } http://git-wip-us.apache.org/repos/asf/cassandra/blob/4f1e5e28/src/java/org/apache/cassandra/db/IColumnContainer.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/db/IColumnContainer.java index 27123ee,0635387..a3bd210 --- a/src/java/org/apache/cassandra/db/IColumnContainer.java +++ b/src/java/org/apache/cassandra/db/IColumnContainer.java @@@ -40,8 -43,8 +40,8 @@@ public interface IColumnContaine public boolean replace(IColumn oldColumn, IColumn newColumn); public boolean isMarkedForDelete(); - public long getMarkedForDeleteAt(); + public DeletionInfo deletionInfo(); - public boolean hasExpiredTombstones(int gcBefore); + public boolean hasIrrelevantData(int gcBefore); public AbstractType<?> getComparator(); http://git-wip-us.apache.org/repos/asf/cassandra/blob/4f1e5e28/src/java/org/apache/cassandra/db/compaction/CompactionIterable.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/4f1e5e28/src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java index b7c6883,8ed21ca..7d4ae4b --- a/src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java +++ b/src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java @@@ -61,9 -67,8 +61,9 @@@ public class PrecompactedRow extends Ab // taking this into account. Boolean shouldPurge = null; - if (cf.hasExpiredTombstones(controller.gcBefore)) + if (cf.hasIrrelevantData(controller.gcBefore)) shouldPurge = controller.shouldPurge(key); + // We should only gc tombstone if shouldPurge == true. But otherwise, // it is still ok to collect column that shadowed by their (deleted) // container, which removeDeleted(cf, Integer.MAX_VALUE) will do
