You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2012/07/02 10:14:09 UTC

[1/10] git commit: merge from 1.1

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 <jb...@apache.org>
Authored: Mon Jul 2 03:05:47 2012 -0500
Committer: Jonathan Ellis <jb...@apache.org>
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